You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/02/13 03:15:16 UTC

[GitHub] merlimat closed pull request #1150: PIP-10: Removing cluster from topic name

merlimat closed pull request #1150: PIP-10: Removing cluster from topic name
URL: https://github.com/apache/incubator-pulsar/pull/1150
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 51e6acec3..9a2b44a74 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
@@ -275,7 +275,8 @@ public void start() throws PulsarServerException {
 
             this.webService = new WebService(this);
             this.webService.addRestResources("/", "org.apache.pulsar.broker.web", false);
-            this.webService.addRestResources("/admin", "org.apache.pulsar.broker.admin", true);
+            this.webService.addRestResources("/admin", "org.apache.pulsar.broker.admin.v1", true);
+            this.webService.addRestResources("/admin/v2", "org.apache.pulsar.broker.admin.v2", true);
             this.webService.addRestResources("/lookup", "org.apache.pulsar.broker.lookup", true);
 
             this.webService.addServlet("/metrics",
@@ -462,8 +463,7 @@ public void loadNamespaceDestinations(NamespaceBundle bundle) {
             List<CompletableFuture<Topic>> 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 f04d15bb6..b91c63ac6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
@@ -52,6 +52,7 @@
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
 import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies;
+import org.apache.pulsar.common.util.Codec;
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.apache.pulsar.zookeeper.ZooKeeperCache;
 import org.apache.pulsar.zookeeper.ZooKeeperCache.Deserializer;
@@ -189,11 +190,22 @@ public void validatePoliciesReadOnlyAccess() {
     protected List<String> getListOfNamespaces(String property) throws Exception {
         List<String> 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<String> children = globalZk().getChildren(path(POLICIES, property, clusterOrNamespace), false);
+                if (children == null || children.isEmpty()) {
+                    String namespace = NamespaceName.get(property, clusterOrNamespace).toString();
+                    // if the length is 0 then this is probably a leftover cluster from namespace created
+                    // with the v1 admin format (prop/cluster/ns) and then deleted, so no need to add it to the list
+                    if (globalZk().getData(path(POLICIES, namespace), false, null).length != 0) {
+                        namespaces.add(namespace);
+                    }
+                } else {
+                    children.forEach(ns -> {
+                        namespaces.add(NamespaceName.get(property, clusterOrNamespace, ns).toString());
+                    });
                 }
             } catch (KeeperException.NoNodeException e) {
                 // A cluster was deleted between the 2 getChildren() calls, ignoring
@@ -204,7 +216,56 @@ public void validatePoliciesReadOnlyAccess() {
         return namespaces;
     }
 
-    
+    protected NamespaceName namespaceName;
+
+    protected void validateNamespaceName(String property, String namespace) {
+        try {
+            this.namespaceName = NamespaceName.get(property, namespace);
+        } catch (IllegalArgumentException e) {
+            log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), namespace, e);
+            throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid");
+        }
+    }
+
+    @Deprecated
+    protected void validateNamespaceName(String property, String cluster, String namespace) {
+        try {
+            this.namespaceName = NamespaceName.get(property, cluster, namespace);
+        } catch (IllegalArgumentException e) {
+            log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), namespace, e);
+            throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid");
+        }
+    }
+
+    protected DestinationName destinationName;
+
+    protected void validateDestinationName(String property, String namespace, String encodedTopic) {
+        String topic = Codec.decode(encodedTopic);
+        try {
+            this.namespaceName = NamespaceName.get(property, namespace);
+            this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+        } catch (IllegalArgumentException e) {
+            log.warn("[{}] Failed to validate topic name {}://{}/{}/{}", clientAppId(), domain(), property, namespace,
+                    topic, e);
+            throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid");
+        }
+
+        this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+    }
+
+    @Deprecated
+    protected void validateDestinationName(String property, String cluster, String namespace, String encodedTopic) {
+        String topic = Codec.decode(encodedTopic);
+        try {
+            this.namespaceName = NamespaceName.get(property, cluster, namespace);
+            this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+        } catch (IllegalArgumentException e) {
+            log.warn("[{}] Failed to validate topic name {}://{}/{}/{}/{}", clientAppId(), domain(), property, cluster,
+                    namespace, topic, e);
+            throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid");
+        }
+    }
+
     /**
      * Redirect the call to the specified broker
      *
@@ -227,20 +288,20 @@ protected void validateBrokerName(String broker) throws MalformedURLException {
         }
     }
 
-    protected Policies getNamespacePolicies(String property, String cluster, String namespace) {
+    protected Policies getNamespacePolicies(NamespaceName namespaceName) {
         try {
-            Policies policies = policiesCache().get(AdminResource.path(POLICIES, property, cluster, namespace))
+            Policies policies = policiesCache().get(AdminResource.path(POLICIES, namespaceName.toString()))
                     .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist"));
             // fetch bundles from LocalZK-policies
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory()
-                    .getBundles(NamespaceName.get(property, cluster, namespace));
+                    .getBundles(namespaceName);
             BundlesData bundleData = NamespaceBundleFactory.getBundlesData(bundles);
             policies.bundles = bundleData != null ? bundleData : policies.bundles;
             return policies;
         } catch (RestException re) {
             throw re;
         } catch (Exception e) {
-            log.error("[{}] Failed to get namespace policies {}/{}/{}", clientAppId(), property, cluster, namespace, e);
+            log.error("[{}] Failed to get namespace policies {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
     }
@@ -249,27 +310,27 @@ public static ObjectMapper jsonMapper() {
         return ObjectMapperFactory.getThreadLocal();
     }
 
-    ZooKeeperDataCache<PropertyAdmin> propertiesCache() {
+    public ZooKeeperDataCache<PropertyAdmin> propertiesCache() {
         return pulsar().getConfigurationCache().propertiesCache();
     }
 
-    ZooKeeperDataCache<Policies> policiesCache() {
+    protected ZooKeeperDataCache<Policies> policiesCache() {
         return pulsar().getConfigurationCache().policiesCache();
     }
 
-    ZooKeeperDataCache<LocalPolicies> localPoliciesCache() {
+    protected ZooKeeperDataCache<LocalPolicies> localPoliciesCache() {
         return pulsar().getLocalZkCacheService().policiesCache();
     }
 
-    ZooKeeperDataCache<ClusterData> clustersCache() {
+    protected ZooKeeperDataCache<ClusterData> clustersCache() {
         return pulsar().getConfigurationCache().clustersCache();
     }
 
-    ZooKeeperChildrenCache managedLedgerListCache() {
+    protected ZooKeeperChildrenCache managedLedgerListCache() {
         return pulsar().getLocalZkCacheService().managedLedgerListCache();
     }
 
-    Set<String> clusters() {
+    protected Set<String> clusters() {
         try {
             return pulsar().getConfigurationCache().clustersListCache().get();
         } catch (Exception e) {
@@ -277,7 +338,7 @@ ZooKeeperChildrenCache managedLedgerListCache() {
         }
     }
 
-    ZooKeeperChildrenCache clustersListCache() {
+    protected ZooKeeperChildrenCache clustersListCache() {
         return pulsar().getConfigurationCache().clustersListCache();
     }
 
@@ -297,32 +358,30 @@ protected ZooKeeperChildrenCache failureDomainListCache() {
         return pulsar().getConfigurationCache().failureDomainListCache();
     }
 
-    protected PartitionedTopicMetadata getPartitionedTopicMetadata(String property, String cluster, String namespace,
-            String destination, boolean authoritative) {
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateClusterOwnership(dn.getCluster());
+    protected PartitionedTopicMetadata getPartitionedTopicMetadata(DestinationName destinationName,
+            boolean authoritative) {
+        validateClusterOwnership(destinationName.getCluster());
         // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
         // serve/redirect request else fail partitioned-metadata-request so, client fails while creating
         // producer/consumer
-        validateGlobalNamespaceOwnership(dn.getNamespaceObject());
-        
+        validateGlobalNamespaceOwnership(destinationName.getNamespaceObject());
+
         try {
-            checkConnect(dn);
+            checkConnect(destinationName);
         } catch (WebApplicationException e) {
-            validateAdminAccessOnProperty(dn.getProperty());
+            validateAdminAccessOnProperty(destinationName.getProperty());
         } catch (Exception e) {
             // unknown error marked as internal server error
-            log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destination,
+            log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destinationName,
                     clientAppId(), e.getMessage(), e);
             throw new RestException(e);
         }
 
-        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(),
-                dn.getEncodedLocalName());
+        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), destinationName.getEncodedLocalName());
         PartitionedTopicMetadata partitionMetadata = fetchPartitionedTopicMetadata(pulsar(), path);
 
         if (log.isDebugEnabled()) {
-            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), dn,
+            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), destinationName,
                     partitionMetadata.partitions);
         }
         return partitionMetadata;
@@ -339,8 +398,8 @@ protected static PartitionedTopicMetadata fetchPartitionedTopicMetadata(PulsarSe
         }
     }
 
-    protected static CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadataAsync(PulsarService pulsar,
-            String path) {
+    protected static CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadataAsync(
+            PulsarService pulsar, String path) {
         CompletableFuture<PartitionedTopicMetadata> metadataFuture = new CompletableFuture<>();
         try {
             // gets the number of partitions from the zk cache
@@ -375,4 +434,22 @@ protected void validateClusterExists(String cluster) {
             throw new RestException(e);
         }
     }
+
+    protected Policies getNamespacePolicies(String property, String cluster, String namespace) {
+        try {
+            Policies policies = policiesCache().get(AdminResource.path(POLICIES, property, cluster, namespace))
+                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist"));
+            // fetch bundles from LocalZK-policies
+            NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory()
+                    .getBundles(NamespaceName.get(property, cluster, namespace));
+            BundlesData bundleData = NamespaceBundleFactory.getBundlesData(bundles);
+            policies.bundles = bundleData != null ? bundleData : policies.bundles;
+            return policies;
+        } catch (RestException re) {
+            throw re;
+        } catch (Exception e) {
+            log.error("[{}] Failed to get namespace policies {}/{}/{}", clientAppId(), property, cluster, namespace, e);
+            throw new RestException(e);
+        }
+    }
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
deleted file mode 100644
index e2467f90b..000000000
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
+++ /dev/null
@@ -1,1757 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.pulsar.broker.admin;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.commons.lang3.StringUtils.isBlank;
-import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
-import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES_ROOT;
-import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT;
-
-import java.net.URI;
-import java.net.URL;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Optional;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.Status;
-import javax.ws.rs.core.UriBuilder;
-
-import org.apache.pulsar.broker.PulsarServerException;
-import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
-import org.apache.pulsar.broker.service.Subscription;
-import org.apache.pulsar.broker.service.Topic;
-import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
-import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationName;
-import org.apache.pulsar.common.naming.NamedEntity;
-import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceBundleFactory;
-import org.apache.pulsar.common.naming.NamespaceBundles;
-import org.apache.pulsar.common.naming.NamespaceName;
-import org.apache.pulsar.common.policies.data.AuthAction;
-import org.apache.pulsar.common.policies.data.BacklogQuota;
-import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
-import org.apache.pulsar.common.policies.data.BundlesData;
-import org.apache.pulsar.common.policies.data.ClusterData;
-import org.apache.pulsar.common.policies.data.DispatchRate;
-import org.apache.pulsar.common.policies.data.PersistencePolicies;
-import org.apache.pulsar.common.policies.data.Policies;
-import org.apache.pulsar.common.policies.data.RetentionPolicies;
-import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
-import org.apache.pulsar.common.util.FutureUtil;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.collect.Sets.SetView;
-
-import io.swagger.annotations.Api;
-import io.swagger.annotations.ApiOperation;
-import io.swagger.annotations.ApiResponse;
-import io.swagger.annotations.ApiResponses;
-
-@Path("/namespaces")
-@Produces(MediaType.APPLICATION_JSON)
-@Consumes(MediaType.APPLICATION_JSON)
-@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
-public class Namespaces extends AdminResource {
-
-    public static final String GLOBAL_CLUSTER = "global";
-    private static final long MAX_BUNDLES = ((long) 1) << 32;
-
-    @GET
-    @Path("/{property}")
-    @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property doesn't exist") })
-    public List<String> getPropertyNamespaces(@PathParam("property") String property) {
-        validateAdminAccessOnProperty(property);
-
-        try {
-            return getListOfNamespaces(property);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to get namespace list for propery: {} - Does not exist", clientAppId(), property);
-            throw new RestException(Status.NOT_FOUND, "Property does not exist");
-        } catch (Exception e) {
-            log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}")
-    @ApiOperation(value = "Get the list of all the namespaces for a certain property on single cluster.", response = String.class, responseContainer = "Set")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster doesn't exist") })
-    public List<String> getNamespacesForCluster(@PathParam("property") String property,
-            @PathParam("cluster") String cluster) {
-        validateAdminAccessOnProperty(property);
-        List<String> namespaces = Lists.newArrayList();
-        if (!clusters().contains(cluster)) {
-            log.warn("[{}] Failed to get namespace list for property: {}/{} - Cluster does not exist", clientAppId(),
-                    property, cluster);
-            throw new RestException(Status.NOT_FOUND, "Cluster does not exist");
-        }
-
-        try {
-            for (String namespace : globalZk().getChildren(path(POLICIES, property, cluster), false)) {
-                namespaces.add(String.format("%s/%s/%s", property, cluster, namespace));
-            }
-        } catch (KeeperException.NoNodeException e) {
-            // NoNode means there are no namespaces for this property on the specified cluster, returning empty list
-        } catch (Exception e) {
-            log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e);
-            throw new RestException(e);
-        }
-
-        namespaces.sort(null);
-        return namespaces;
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/destinations")
-    @ApiOperation(value = "Get the list of all the destinations under a certain namespace.", response = String.class, responseContainer = "Set")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public List<String> getDestinations(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        // Validate that namespace exists, throws 404 if it doesn't exist
-        getNamespacePolicies(property, cluster, namespace);
-
-        try {
-            return pulsar().getNamespaceService().getListOfDestinations(property, cluster, namespace);
-        } catch (Exception e) {
-            log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}")
-    @ApiOperation(value = "Get the dump all the policies specified for a namespace.", response = Policies.class)
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public Policies getPolicies(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        return getNamespacePolicies(property, cluster, namespace);
-    }
-
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}")
-    @ApiOperation(value = "Creates a new empty namespace with no policies attached.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Namespace already exists"),
-            @ApiResponse(code = 412, message = "Namespace name is not valid") })
-    public void createNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, BundlesData initialBundles) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-        // If the namespace is non global, make sure property has the access on the cluster. For global namespace, same
-        // check is made at the time of setting replication.
-        if (!cluster.equals(GLOBAL_CLUSTER)) {
-            validateClusterForProperty(property, cluster);
-        }
-        if (!clusters().contains(cluster)) {
-            log.warn("[{}] Failed to create namespace. Cluster {} does not exist", clientAppId(), cluster);
-            throw new RestException(Status.NOT_FOUND, "Cluster does not exist");
-        }
-        try {
-            checkNotNull(propertiesCache().get(path(POLICIES, property)));
-        } catch (NoNodeException nne) {
-            log.warn("[{}] Failed to create namespace. Property {} does not exist", clientAppId(), property);
-            throw new RestException(Status.NOT_FOUND, "Property does not exist");
-        } catch (RestException e) {
-            throw e;
-        } catch (Exception e) {
-            throw new RestException(e);
-        }
-        try {
-            NamedEntity.checkName(namespace);
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            Policies policies = new Policies();
-            if (initialBundles != null && initialBundles.getNumBundles() > 0) {
-                if (initialBundles.getBoundaries() == null || initialBundles.getBoundaries().size() == 0) {
-                    policies.bundles = getBundles(initialBundles.getNumBundles());
-                } else {
-                    policies.bundles = validateBundlesData(initialBundles);
-                }
-            } else {
-                int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles();
-                policies.bundles = getBundles(defaultNumberOfBundles);
-            }
-
-            zkCreateOptimistic(path(POLICIES, property, cluster, namespace),
-                    jsonMapper().writeValueAsBytes(policies));
-            log.info("[{}] Created namespace {}/{}/{}", clientAppId(), property, cluster, namespace);
-        } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create namespace {}/{}/{} - already exists", clientAppId(), property, cluster,
-                    namespace);
-            throw new RestException(Status.CONFLICT, "Namespace already exists");
-        } catch (IllegalArgumentException e) {
-            log.warn("[{}] Failed to create namespace with invalid name {}", clientAppId(), property, e);
-            throw new RestException(Status.PRECONDITION_FAILED, "Namespace name is not valid");
-        } catch (Exception e) {
-            log.error("[{}] Failed to create namespace {}/{}/{}", clientAppId(), property, cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}")
-    @ApiOperation(value = "Delete a namespace and all the destinations under it.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Namespace is not empty") })
-    public void deleteNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        // ensure that non-global namespace is directed to the correct cluster
-        validateClusterOwnership(cluster);
-
-        Entry<Policies, Stat> 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<String> destinations = getDestinations(property, cluster, namespace);
-        if (!destinations.isEmpty()) {
-            log.info("Found destinations: {}", destinations);
-            throw new RestException(Status.CONFLICT, "Cannot delete non empty namespace");
-        }
-
-        // set the policies to deleted so that somebody else cannot acquire this namespace
-        try {
-            policies.deleted = true;
-            globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies),
-                    policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-        } catch (Exception e) {
-            log.error("[{}] Failed to delete namespace on global ZK {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
-            throw new RestException(e);
-        }
-
-        // remove from owned namespace map and ephemeral node from ZK
-        try {
-            NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
-            for (NamespaceBundle bundle : bundles.getBundles()) {
-                // check if the bundle is owned by any broker, if not then we do not need to delete the bundle
-                if (pulsar().getNamespaceService().getOwner(bundle).isPresent()) {
-                    pulsar().getAdminClient().namespaces().deleteNamespaceBundle(nsName.toString(),
-                            bundle.getBundleRange());
-                }
-            }
-
-            // we have successfully removed all the ownership for the namespace, the policies znode can be deleted now
-            final String globalZkPolicyPath = path(POLICIES, property, cluster, namespace);
-            final String lcaolZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, property, cluster, namespace);
-            globalZk().delete(globalZkPolicyPath, -1);
-            localZk().delete(lcaolZkPolicyPath, -1);
-            policiesCache().invalidate(globalZkPolicyPath);
-            localCacheService().policiesCache().invalidate(lcaolZkPolicyPath);
-        } catch (PulsarAdminException cae) {
-            throw new RestException(cae);
-        } catch (Exception e) {
-            log.error(String.format("[%s] Failed to remove owned namespace %s/%s/%s", clientAppId(), property, cluster,
-                    namespace), e);
-            // avoid throwing exception in case of the second failure
-        }
-
-    }
-
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{bundle}")
-    @ApiOperation(value = "Delete a namespace bundle and all the destinations under it.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Namespace bundle is not empty") })
-    public void deleteNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        // ensure that non-global namespace is directed to the correct cluster
-        validateClusterOwnership(cluster);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        // ensure the local cluster is the only cluster for the global namespace configuration
-        try {
-            if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-                if (policies.replication_clusters.size() > 1) {
-                    // There are still more than one clusters configured for the global namespace
-                    throw new RestException(Status.PRECONDITION_FAILED, "Cannot delete the global namespace " + nsName
-                            + ". There are still more than one replication clusters configured.");
-                }
-                if (policies.replication_clusters.size() == 1
-                        && !policies.replication_clusters.contains(config().getClusterName())) {
-                    // the only replication cluster is other cluster, redirect
-                    String replCluster = policies.replication_clusters.get(0);
-                    ClusterData replClusterData = clustersCache().get(AdminResource.path("clusters", replCluster))
-                            .orElseThrow(() -> new RestException(Status.NOT_FOUND,
-                                    "Cluser " + replCluster + " does not exist"));
-                    URL replClusterUrl;
-                    if (!config().isTlsEnabled()) {
-                        replClusterUrl = new URL(replClusterData.getServiceUrl());
-                    } else if (!replClusterData.getServiceUrlTls().isEmpty()) {
-                        replClusterUrl = new URL(replClusterData.getServiceUrlTls());
-                    } else {
-                        throw new RestException(Status.PRECONDITION_FAILED,
-                                "The replication cluster does not provide TLS encrypted service");
-                    }
-                    URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(replClusterUrl.getHost())
-                            .port(replClusterUrl.getPort()).replaceQueryParam("authoritative", false).build();
-                    log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, cluster);
-                    throw new WebApplicationException(Response.temporaryRedirect(redirect).build());
-                }
-            }
-        } catch (WebApplicationException wae) {
-            throw wae;
-        } catch (Exception e) {
-            throw new RestException(e);
-        }
-
-        NamespaceBundle bundle = validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative,
-                true);
-        try {
-            List<String> destinations = getDestinations(property, cluster, namespace);
-            for (String destination : destinations) {
-                NamespaceBundle destinationBundle = (NamespaceBundle) pulsar().getNamespaceService()
-                        .getBundle(DestinationName.get(destination));
-                if (bundle.equals(destinationBundle)) {
-                    throw new RestException(Status.CONFLICT, "Cannot delete non empty bundle");
-                }
-            }
-
-            // remove from owned namespace map and ephemeral node from ZK
-            pulsar().getNamespaceService().removeOwnedServiceUnit(bundle);
-        } catch (WebApplicationException wae) {
-            throw wae;
-        } catch (Exception e) {
-            log.error("[{}] Failed to remove namespace bundle {}/{}", clientAppId(), nsName.toString(), bundleRange, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/permissions")
-    @ApiOperation(value = "Retrieve the permissions for a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Namespace is not empty") })
-    public Map<String, Set<AuthAction>> getPermissions(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        return policies.auth_policies.namespace_auth;
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/permissions/{role}")
-    @ApiOperation(value = "Grant a new permission to a role on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void grantPermissionOnNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("role") String role, Set<AuthAction> actions) {
-        validateAdminAccessOnProperty(property);
-
-        NamespaceName namespaceName = NamespaceName.get(property, cluster, namespace);
-        try {
-            pulsar().getBrokerService().getAuthorizationService()
-                    .grantPermissionAsync(namespaceName, actions, role, null/*additional auth-data json*/)
-                    .get();
-        } catch (InterruptedException e) {
-            log.error("[{}] Failed to get permissions for namespace {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
-            throw new RestException(e);
-        } catch (ExecutionException e) {
-            if (e.getCause() instanceof IllegalArgumentException) {
-                log.warn("[{}] Failed to set permissions for namespace {}/{}/{}: does not exist", clientAppId(),
-                        property, cluster, namespace);
-                throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-            } else if (e.getCause() instanceof IllegalStateException) {
-                log.warn("[{}] Failed to set permissions for namespace {}/{}/{}: concurrent modification",
-                        clientAppId(), property, cluster, namespace);
-                throw new RestException(Status.CONFLICT, "Concurrent modification");
-            } else {
-                log.error("[{}] Failed to get permissions for namespace {}/{}/{}", clientAppId(), property, cluster,
-                        namespace, e);
-                throw new RestException(e);
-            }
-        }
-    }
-
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/permissions/{role}")
-    @ApiOperation(value = "Revoke all permissions to a role on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public void revokePermissionsOnNamespace(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("role") String role) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        try {
-            Stat nodeStat = new Stat();
-            byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            policies.auth_policies.namespace_auth.remove(role);
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies),
-                    nodeStat.getVersion());
-
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully revoked access for role {} - namespace {}/{}/{}", clientAppId(), role, property,
-                    cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to revoke permissions for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to revoke permissions on namespace {}/{}/{}: concurrent modification", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to revoke permissions on namespace {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/replication")
-    @ApiOperation(value = "Get the replication clusters for a namespace.", response = String.class, responseContainer = "List")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Namespace is not global") })
-    public List<String> getNamespaceReplicationClusters(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        if (!cluster.equals("global")) {
-            throw new RestException(Status.PRECONDITION_FAILED,
-                    "Cannot get the replication clusters for a non-global namespace");
-        }
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        return policies.replication_clusters;
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/replication")
-    @ApiOperation(value = "Set the replication clusters for a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"),
-            @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") })
-    public void setNamespaceReplicationClusters(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, List<String> clusterIds) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        Set<String> 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<String> 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<Policies, Stat> policiesNode = null;
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-
-        try {
-            // Force to read the data s.t. the watch to the cache content is setup.
-            policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace))
-                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist"));
-            policiesNode.getKey().replication_clusters = clusterIds;
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace),
-                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-
-            log.info("[{}] Successfully updated the replication clusters on namespace {}/{}/{}", clientAppId(),
-                    property, cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update the replication clusters for namespace {}/{}/{}: does not exist",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn(
-                    "[{}] Failed to update the replication clusters on namespace {}/{}/{} expected policy node version={} : concurrent modification",
-                    clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion());
-
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to update the replication clusters on namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/messageTTL")
-    @ApiOperation(value = "Get the message TTL for the namespace")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public int getNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        return policies.message_ttl_in_seconds;
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/messageTTL")
-    @ApiOperation(value = "Set message TTL in seconds for namespace")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Invalid TTL") })
-    public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, int messageTTL) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        if (messageTTL < 0) {
-            throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL");
-        }
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        Entry<Policies, Stat> policiesNode = null;
-
-        try {
-            // Force to read the data s.t. the watch to the cache content is setup.
-            policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace))
-                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist"));
-            policiesNode.getKey().message_ttl_in_seconds = messageTTL;
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace),
-                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-
-            log.info("[{}] Successfully updated the message TTL on namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update the message TTL for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn(
-                    "[{}] Failed to update the message TTL on namespace {}/{}/{} expected policy node version={} : concurrent modification",
-                    clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion());
-
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to update the message TTL on namespace {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/antiAffinity")
-    @ApiOperation(value = "Set anti-affinity group for a namespace")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Invalid antiAffinityGroup") })
-    public void setNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, String antiAffinityGroup) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-        
-        log.info("[{}] Setting anti-affinity group {} for {}/{}/{}", clientAppId(), antiAffinityGroup, property,
-                cluster, namespace);
-
-        if (isBlank(antiAffinityGroup)) {
-            throw new RestException(Status.PRECONDITION_FAILED, "antiAffinityGroup can't be empty");
-        }
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        Entry<Policies, Stat> 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<String> 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<String> namespaces = Lists.newArrayList();
-        try {
-            for (String prop : globalZk().getChildren(POLICIES_ROOT, false)) {
-                for (String namespace : globalZk().getChildren(path(POLICIES, prop, cluster), false)) {
-                    Optional<Policies> policies = policiesCache()
-                            .get(AdminResource.path(POLICIES, prop, cluster, namespace));
-                    if (policies.isPresent() && antiAffinityGroup.equalsIgnoreCase(policies.get().antiAffinityGroup)) {
-                        namespaces.add(String.format("%s/%s/%s", prop, cluster, namespace));
-                    }
-                }
-            }
-        } catch (Exception e) {
-            log.warn("Failed to list of properties/namespace from global-zk", e);
-        }
-        return namespaces;
-    }
-    
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/antiAffinity")
-    @ApiOperation(value = "Remove anti-affinity group of a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void removeNamespaceAntiAffinityGroup(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        log.info("[{}] Deleting anti-affinity group for {}/{}/{}", clientAppId(), property, cluster, namespace);
-        
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            policies.antiAffinityGroup = null;
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully removed anti-affinity group for a namespace={}/{}/{}", clientAppId(), property,
-                    cluster, namespace);
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/deduplication")
-    @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public void modifyDeduplication(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, boolean enableDeduplication) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        Entry<Policies, Stat> policiesNode = null;
-
-        try {
-            // Force to read the data s.t. the watch to the cache content is setup.
-            policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace))
-                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist"));
-            policiesNode.getKey().deduplicationEnabled = enableDeduplication;
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace),
-                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-
-            log.info("[{}] Successfully {} on namespace {}/{}/{}", clientAppId(),
-                    enableDeduplication ? "enabled" : "disabled", property, cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to modify deplication status for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn(
-                    "[{}] Failed to modify deplication status on namespace {}/{}/{} expected policy node version={} : concurrent modification",
-                    clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion());
-
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to modify deplication status on namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/bundles")
-    @ApiOperation(value = "Get the bundles split data.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") })
-    public BundlesData getBundlesData(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        return policies.bundles;
-    }
-
-    private BundlesData validateBundlesData(BundlesData initialBundles) {
-        SortedSet<String> partitions = new TreeSet<String>();
-        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<String> 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<String> partitions = Lists.newArrayList();
-        partitions.add(String.format("0x%08x", 0l));
-        Long curPartition = segSize;
-        for (int i = 0; i < numBundles; i++) {
-            if (i != numBundles - 1) {
-                partitions.add(String.format("0x%08x", curPartition));
-            } else {
-                partitions.add(String.format("0x%08x", maxVal - 1));
-            }
-            curPartition += segSize;
-        }
-        return new BundlesData(partitions);
-    }
-
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/unload")
-    @ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
-            + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
-            + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
-            + "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
-            + "result in non-persistent message loss and unexpected connection closure to the clients.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") })
-    public void unloadNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        log.info("[{}] Unloading namespace {}/{}/{}", clientAppId(), property, cluster, namespace);
-
-        validateSuperUserAccess();
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-
-        List<String> boundaries = policies.bundles.getBoundaries();
-        for (int i = 0; i < boundaries.size() - 1; i++) {
-            String bundle = String.format("%s_%s", boundaries.get(i), boundaries.get(i + 1));
-            try {
-                pulsar().getAdminClient().namespaces().unloadNamespaceBundle(nsName.toString(), bundle);
-            } catch (PulsarServerException | PulsarAdminException e) {
-                log.error(String.format("[%s] Failed to unload namespace %s/%s/%s", clientAppId(), property, cluster,
-                        namespace), e);
-                throw new RestException(e);
-            }
-        }
-        log.info("[{}] Successfully unloaded all the bundles in namespace {}/{}/{}", clientAppId(), property, cluster,
-                namespace);
-    }
-
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/{bundle}/unload")
-    @ApiOperation(value = "Unload a namespace bundle")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
-    public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        log.info("[{}] Unloading namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace,
-                bundleRange);
-
-        validateSuperUserAccess();
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
-        validatePoliciesReadOnlyAccess();
-
-        if (!isBundleOwnedByAnyBroker(fqnn, policies.bundles, bundleRange)) {
-            log.info("[{}] Namespace bundle is not owned by any broker {}/{}/{}/{}", clientAppId(), property, cluster,
-                    namespace, bundleRange);
-            return;
-        }
-
-        NamespaceBundle nsBundle = validateNamespaceBundleOwnership(fqnn, policies.bundles, bundleRange, authoritative,
-                true);
-        try {
-            pulsar().getNamespaceService().unloadNamespaceBundle(nsBundle);
-            log.info("[{}] Successfully unloaded namespace bundle {}", clientAppId(), nsBundle.toString());
-        } catch (Exception e) {
-            log.error("[{}] Failed to unload namespace bundle {}/{}", clientAppId(), fqnn.toString(), bundleRange, e);
-            throw new RestException(e);
-        }
-    }
-
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/{bundle}/split")
-    @ApiOperation(value = "Split a namespace bundle")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
-    public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative,
-            @QueryParam("unload") @DefaultValue("false") boolean unload) {
-        log.info("[{}] Split namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace, bundleRange);
-
-        validateSuperUserAccess();
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
-        validatePoliciesReadOnlyAccess();
-        NamespaceBundle nsBundle = validateNamespaceBundleOwnership(fqnn, policies.bundles, bundleRange, authoritative,
-                true);
-
-        try {
-            pulsar().getNamespaceService().splitAndOwnBundle(nsBundle, unload).get();
-            log.info("[{}] Successfully split namespace bundle {}", clientAppId(), nsBundle.toString());
-        } catch (IllegalArgumentException e) {
-            log.error("[{}] Failed to split namespace bundle {}/{} due to {}", clientAppId(), fqnn.toString(),
-                    bundleRange, e.getMessage());
-            throw new RestException(Status.PRECONDITION_FAILED, "Split bundle failed due to invalid request");
-        } catch (Exception e) {
-            log.error("[{}] Failed to split namespace bundle {}/{}", clientAppId(), fqnn.toString(), bundleRange, e);
-            throw new RestException(e);
-        }
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/dispatchRate")
-    @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
-    public void setDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, DispatchRate dispatchRate) {
-        log.info("[{}] Set namespace dispatch-rate {}/{}/{}/{}", clientAppId(), property, cluster, namespace,
-                dispatchRate);
-        validateSuperUserAccess();
-
-        Entry<Policies, Stat> policiesNode = null;
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-
-        try {
-            final String path = path(POLICIES, property, cluster, namespace);
-            // Force to read the data s.t. the watch to the cache content is setup.
-            policiesNode = policiesCache().getWithStat(path)
-                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist"));
-            policiesNode.getKey().clusterDispatchRate.put(pulsar().getConfiguration().getClusterName(), dispatchRate);
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policiesNode.getKey()),
-                    policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path);
-
-            log.info("[{}] Successfully updated the dispatchRate for cluster on namespace {}/{}/{}", clientAppId(),
-                    property, cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{}: does not exist",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn(
-                    "[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{} expected policy node version={} : concurrent modification",
-                    clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion());
-
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to update the dispatchRate for cluster on namespace {}/{}/{}", clientAppId(),
-                    property, cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/dispatchRate")
-    @ApiOperation(value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public DispatchRate getDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        DispatchRate dispatchRate = policies.clusterDispatchRate.get(pulsar().getConfiguration().getClusterName());
-        if (dispatchRate != null) {
-            return dispatchRate;
-        } else {
-            throw new RestException(Status.NOT_FOUND,
-                    "Dispatch-rate is not configured for cluster " + pulsar().getConfiguration().getClusterName());
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/backlogQuotaMap")
-    @ApiOperation(value = "Get backlog quota map on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public Map<BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        return policies.backlog_quota_map;
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/backlogQuota")
-    @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification"),
-            @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") })
-    public void setBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType,
-            BacklogQuota backlogQuota) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        if (backlogQuotaType == null) {
-            backlogQuotaType = BacklogQuotaType.destination_storage;
-        }
-
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            RetentionPolicies r = policies.retention_policies;
-            if (r != null) {
-                Policies p = new Policies();
-                p.backlog_quota_map.put(backlogQuotaType, backlogQuota);
-                if (!checkQuotas(p, r)) {
-                    log.warn(
-                            "[{}] Failed to update backlog configuration for namespace {}/{}/{}: conflicts with retention quota",
-                            clientAppId(), property, cluster, namespace);
-                    throw new RestException(Status.PRECONDITION_FAILED,
-                            "Backlog Quota exceeds configured retention quota for namespace. Please increase retention quota and retry");
-                }
-            }
-            policies.backlog_quota_map.put(backlogQuotaType, backlogQuota);
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully updated backlog quota map: namespace={}/{}/{}, map={}", clientAppId(), property,
-                    cluster, namespace, jsonMapper().writeValueAsString(policies.backlog_quota_map));
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (RestException pfe) {
-            throw pfe;
-        } catch (Exception e) {
-            log.error("[{}] Failed to update backlog quota map for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/backlogQuota")
-    @ApiOperation(value = "Remove a backlog quota policy from a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void removeBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace,
-            @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        if (backlogQuotaType == null) {
-            backlogQuotaType = BacklogQuotaType.destination_storage;
-        }
-
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            policies.backlog_quota_map.remove(backlogQuotaType);
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully removed backlog namespace={}/{}/{}, quota={}", clientAppId(), property, cluster,
-                    namespace, backlogQuotaType);
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to update backlog quota map for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to update backlog quota map for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/retention")
-    @ApiOperation(value = "Get retention config on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public RetentionPolicies getRetention(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        if (policies.retention_policies == null) {
-            return new RetentionPolicies(config().getDefaultRetentionTimeInMinutes(),
-                    config().getDefaultRetentionSizeInMB());
-        } else {
-            return policies.retention_policies;
-        }
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/retention")
-    @ApiOperation(value = " Set retention configuration on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification"),
-            @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") })
-    public void setRetention(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, RetentionPolicies retention) {
-        validatePoliciesReadOnlyAccess();
-
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            if (!checkQuotas(policies, retention)) {
-                log.warn(
-                        "[{}] Failed to update retention configuration for namespace {}/{}/{}: conflicts with backlog quota",
-                        clientAppId(), property, cluster, namespace);
-                throw new RestException(Status.PRECONDITION_FAILED,
-                        "Retention Quota must exceed configured backlog quota for namespace.");
-            }
-            policies.retention_policies = retention;
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully updated retention configuration: namespace={}/{}/{}, map={}", clientAppId(),
-                    property, cluster, namespace, jsonMapper().writeValueAsString(policies.retention_policies));
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update retention configuration for namespace {}/{}/{}: does not exist",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to update retention configuration for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (RestException pfe) {
-            throw pfe;
-        } catch (Exception e) {
-            log.error("[{}] Failed to update retention configuration for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-
-    }
-
-    private boolean checkQuotas(Policies policies, RetentionPolicies retention) {
-        Map<BacklogQuota.BacklogQuotaType, BacklogQuota> backlog_quota_map = policies.backlog_quota_map;
-        if (backlog_quota_map.isEmpty() || retention.getRetentionSizeInMB() == 0) {
-            return true;
-        }
-        BacklogQuota quota = backlog_quota_map.get(BacklogQuotaType.destination_storage);
-        if (quota == null) {
-            quota = pulsar().getBrokerService().getBacklogQuotaManager().getDefaultQuota();
-        }
-        if (quota.getLimit() >= ((long) retention.getRetentionSizeInMB() * 1024 * 1024)) {
-            return false;
-        }
-        return true;
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/persistence")
-    @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification"),
-            @ApiResponse(code = 400, message = "Invalid persistence policies") })
-    public void setPersistence(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, PersistencePolicies persistence) {
-        validatePoliciesReadOnlyAccess();
-        validatePersistencePolicies(persistence);
-
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            policies.persistence = persistence;
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully updated persistence configuration: namespace={}/{}/{}, map={}", clientAppId(),
-                    property, cluster, namespace, jsonMapper().writeValueAsString(policies.persistence));
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update persistence configuration for namespace {}/{}/{}: does not exist",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to update persistence configuration for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to update persistence configuration for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    private void validatePersistencePolicies(PersistencePolicies persistence) {
-        try {
-            checkNotNull(persistence);
-            final ServiceConfiguration config = pulsar().getConfiguration();
-            checkArgument(persistence.getBookkeeperEnsemble() <= config.getManagedLedgerMaxEnsembleSize(),
-                    "Bookkeeper-Ensemble must be <= %s", config.getManagedLedgerMaxEnsembleSize());
-            checkArgument(persistence.getBookkeeperWriteQuorum() <= config.getManagedLedgerMaxWriteQuorum(),
-                    "Bookkeeper-WriteQuorum must be <= %s", config.getManagedLedgerMaxWriteQuorum());
-            checkArgument(persistence.getBookkeeperAckQuorum() <= config.getManagedLedgerMaxAckQuorum(),
-                    "Bookkeeper-AckQuorum must be <= %s", config.getManagedLedgerMaxAckQuorum());
-            checkArgument(
-                    (persistence.getBookkeeperEnsemble() >= persistence.getBookkeeperWriteQuorum())
-                            && (persistence.getBookkeeperWriteQuorum() >= persistence.getBookkeeperAckQuorum()),
-                    "Bookkeeper Ensemble (%s) >= WriteQuorum (%s) >= AckQuoru (%s)", persistence.getBookkeeperEnsemble(),
-                    persistence.getBookkeeperWriteQuorum(), persistence.getBookkeeperAckQuorum());
-        }catch(NullPointerException | IllegalArgumentException e) {
-            throw new RestException(Status.PRECONDITION_FAILED, e.getMessage());
-        }
-    }
-
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/persistence")
-    @ApiOperation(value = "Get the persistence configuration for a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public PersistencePolicies getPersistence(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-        if (policies.persistence == null) {
-            return new PersistencePolicies(config().getManagedLedgerDefaultEnsembleSize(),
-                    config().getManagedLedgerDefaultWriteQuorum(), config().getManagedLedgerDefaultAckQuorum(), 0.0d);
-        } else {
-            return policies.persistence;
-        }
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/clearBacklog")
-    @ApiOperation(value = "Clear backlog for all destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        try {
-            NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
-            Exception exception = null;
-            for (NamespaceBundle nsBundle : bundles.getBundles()) {
-                try {
-                    // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to
-                    // clear
-                    if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) {
-                        // TODO: make this admin call asynchronous
-                        pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklog(nsName.toString(),
-                                nsBundle.getBundleRange());
-                    }
-                } catch (Exception e) {
-                    if (exception == null) {
-                        exception = e;
-                    }
-                }
-            }
-            if (exception != null) {
-                if (exception instanceof PulsarAdminException) {
-                    throw new RestException((PulsarAdminException) exception);
-                } else {
-                    throw new RestException(exception.getCause());
-                }
-            }
-        } catch (WebApplicationException wae) {
-            throw wae;
-        } catch (Exception e) {
-            throw new RestException(e);
-        }
-        log.info("[{}] Successfully cleared backlog on all the bundles for namespace {}", clientAppId(),
-                nsName.toString());
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog")
-    @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void clearNamespaceBundleBacklog(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership  for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
-
-        clearBacklog(nsName, bundleRange, null);
-        log.info("[{}] Successfully cleared backlog on namespace bundle {}/{}", clientAppId(), nsName.toString(),
-                bundleRange);
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}")
-    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("subscription") String subscription,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        try {
-            NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
-            Exception exception = null;
-            for (NamespaceBundle nsBundle : bundles.getBundles()) {
-                try {
-                    // check if the bundle is owned by any broker, if not then there is no backlog on this bundle to
-                    // clear
-                    if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) {
-                        // TODO: make this admin call asynchronous
-                        pulsar().getAdminClient().namespaces().clearNamespaceBundleBacklogForSubscription(
-                                nsName.toString(), nsBundle.getBundleRange(), subscription);
-                    }
-                } catch (Exception e) {
-                    if (exception == null) {
-                        exception = e;
-                    }
-                }
-            }
-            if (exception != null) {
-                if (exception instanceof PulsarAdminException) {
-                    throw new RestException((PulsarAdminException) exception);
-                } else {
-                    throw new RestException(exception.getCause());
-                }
-            }
-        } catch (WebApplicationException wae) {
-            throw wae;
-        } catch (Exception e) {
-            throw new RestException(e);
-        }
-        log.info("[{}] Successfully cleared backlog for subscription {} on all the bundles for namespace {}",
-                clientAppId(), subscription, nsName.toString());
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}")
-    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("subscription") String subscription, @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership  for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
-
-        clearBacklog(nsName, bundleRange, subscription);
-        log.info("[{}] Successfully cleared backlog for subscription {} on namespace bundle {}/{}", clientAppId(),
-                subscription, nsName.toString(), bundleRange);
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}")
-    @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        try {
-            NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
-            Exception exception = null;
-            for (NamespaceBundle nsBundle : bundles.getBundles()) {
-                try {
-                    // check if the bundle is owned by any broker, if not then there are no subscriptions
-                    if (pulsar().getNamespaceService().getOwner(nsBundle).isPresent()) {
-                        // TODO: make this admin call asynchronous
-                        pulsar().getAdminClient().namespaces().unsubscribeNamespaceBundle(nsName.toString(),
-                                nsBundle.getBundleRange(), subscription);
-                    }
-                } catch (Exception e) {
-                    if (exception == null) {
-                        exception = e;
-                    }
-                }
-            }
-            if (exception != null) {
-                if (exception instanceof PulsarAdminException) {
-                    throw new RestException((PulsarAdminException) exception);
-                } else {
-                    throw new RestException(exception.getCause());
-                }
-            }
-        } catch (WebApplicationException wae) {
-            throw wae;
-        } catch (Exception e) {
-            throw new RestException(e);
-        }
-        log.info("[{}] Successfully unsubscribed {} on all the bundles for namespace {}", clientAppId(), subscription,
-                nsName.toString());
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}")
-    @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace bundle.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
-            @PathParam("bundle") String bundleRange,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateAdminAccessOnProperty(property);
-
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
-
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
-        } else {
-            // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
-
-        unsubscribe(nsName, bundleRange, subscription);
-        log.info("[{}] Successfully unsubscribed {} on namespace bundle {}/{}", clientAppId(), subscription,
-                nsName.toString(), bundleRange);
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode")
-    @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void setSubscriptionAuthMode(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        if (subscriptionAuthMode == null) {
-            subscriptionAuthMode = SubscriptionAuthMode.None;
-        }
-
-        try {
-            Stat nodeStat = new Stat();
-            final String path = path(POLICIES, property, cluster, namespace);
-            byte[] content = globalZk().getData(path, null, nodeStat);
-            Policies policies = jsonMapper().readValue(content, Policies.class);
-            policies.subscription_auth_mode = subscriptionAuthMode;
-            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-            log.info("[{}] Successfully updated subscription auth mode: namespace={}/{}/{}, map={}", clientAppId(), property,
-                    cluster, namespace, jsonMapper().writeValueAsString(policies.backlog_quota_map));
-
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to update subscription auth mode for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn("[{}] Failed to update subscription auth mode for namespace {}/{}/{}: concurrent modification",
-                    clientAppId(), property, cluster, namespace);
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (RestException pfe) {
-            throw pfe;
-        } catch (Exception e) {
-            log.error("[{}] Failed to update subscription auth mode for namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    private void clearBacklog(NamespaceName nsName, String bundleRange, String subscription) {
-        try {
-            List<Topic> topicList = pulsar().getBrokerService()
-                    .getAllTopicsFromNamespaceBundle(nsName.toString(), nsName.toString() + "/" + bundleRange);
-
-            List<CompletableFuture<Void>> 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<Topic> topicList = pulsar().getBrokerService()
-                    .getAllTopicsFromNamespaceBundle(nsName.toString(), nsName.toString() + "/" + bundleRange);
-            List<CompletableFuture<Void>> 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<String> replicationClusters) {
-        try {
-            ClusterData clusterData = clustersCache().get(path("clusters", clusterName)).orElseThrow(
-                    () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName));
-            Set<String> peerClusters = clusterData.getPeerClusterNames();
-            if (peerClusters != null && !peerClusters.isEmpty()) {
-                SetView<String> conflictPeerClusters = Sets.intersection(peerClusters, replicationClusters);
-                if (!conflictPeerClusters.isEmpty()) {
-                    log.warn("[{}] {}'s peer cluster can't be part of replication clusters {}", clientAppId(),
-                            clusterName, conflictPeerClusters);
-                    throw new RestException(Status.CONFLICT,
-                            String.format("%s's peer-clusters %s can't be part of replication-clusters %s", clusterName,
-                                    conflictPeerClusters, replicationClusters));
-                }
-            }
-        } catch (RestException re) {
-            throw re;
-        } catch (Exception e) {
-            log.warn("[{}] Failed to get cluster-data for {}", clientAppId(), clusterName, e);
-        }
-    }
-    
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/encryptionRequired")
-    @ApiOperation(value = "Message encryption is required or not for all topics in a namespace")
-    @ApiResponses(value = {
-            @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification"),
-    })
-    public void modifyEncryptionRequired(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, boolean encryptionRequired) {
-        validateAdminAccessOnProperty(property);
-        validatePoliciesReadOnlyAccess();
-
-        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
-        Entry<Policies, Stat> policiesNode = null;
-
-        try {
-            // Force to read the data s.t. the watch to the cache content is setup.
-            policiesNode = policiesCache().getWithStat(path(POLICIES, property, cluster, namespace))
-                    .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace " + nsName + " does not exist"));
-            policiesNode.getKey().encryption_required = encryptionRequired;
-
-            // Write back the new policies into zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace),
-                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
-
-            log.info("[{}] Successfully {} on namespace {}/{}/{}", clientAppId(),
-                    encryptionRequired ? "true" : "false", property, cluster, namespace);
-        } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to modify encryption required status for namespace {}/{}/{}: does not exist", clientAppId(),
-                    property, cluster, namespace);
-            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
-        } catch (KeeperException.BadVersionException e) {
-            log.warn(
-                    "[{}] Failed to modify encryption required status on namespace {}/{}/{} expected policy node version={} : concurrent modification",
-                    clientAppId(), property, cluster, namespace, policiesNode.getValue().getVersion());
-
-            throw new RestException(Status.CONFLICT, "Concurrent modification");
-        } catch (Exception e) {
-            log.error("[{}] Failed to modify encryption required status on namespace {}/{}/{}", clientAppId(), property,
-                    cluster, namespace, e);
-            throw new RestException(e);
-        }
-    }
-
-    private static final Logger log = LoggerFactory.getLogger(Namespaces.class);
-}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
similarity index 96%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/BrokerStats.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
index 8fdd31307..549a0332e 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 d4a538e55..8098f97f8 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 612e2e254..733105a37 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Clusters.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+package org.apache.pulsar.broker.admin.impl;
 
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
 
@@ -35,11 +35,10 @@
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.naming.NamedEntity;
@@ -50,7 +49,6 @@
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
@@ -60,17 +58,11 @@
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.google.common.collect.Maps;
 
-import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 
-import static org.apache.pulsar.broker.cache.ConfigurationCacheService.FAILURE_DOMAIN;
-
-@Path("/clusters")
-@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters")
-@Produces(MediaType.APPLICATION_JSON)
-public class Clusters extends AdminResource {
+public class ClustersBase extends AdminResource {
 
     @GET
     @ApiOperation(value = "Get the list of all the Pulsar clusters.", response = String.class, responseContainer = "Set")
@@ -623,6 +615,6 @@ private void validateBrokerExistsInOtherDomain(final String cluster, final Strin
         }
     }
 
-    private static final Logger log = LoggerFactory.getLogger(Clusters.class);
+    private static final Logger log = LoggerFactory.getLogger(ClustersBase.class);
 
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
new file mode 100644
index 000000000..0dda3aed0
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -0,0 +1,1246 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.impl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
+import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT;
+
+import java.net.URI;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.UriBuilder;
+
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.admin.AdminResource;
+import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
+import org.apache.pulsar.broker.service.Subscription;
+import org.apache.pulsar.broker.service.Topic;
+import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.common.naming.NamespaceBundleFactory;
+import org.apache.pulsar.common.naming.NamespaceBundles;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.BacklogQuota;
+import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
+import org.apache.pulsar.common.policies.data.BundlesData;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.DispatchRate;
+import org.apache.pulsar.common.policies.data.PersistencePolicies;
+import org.apache.pulsar.common.policies.data.Policies;
+import org.apache.pulsar.common.policies.data.RetentionPolicies;
+import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.collect.Sets.SetView;
+
+public abstract class NamespacesBase extends AdminResource {
+
+    private static final long MAX_BUNDLES = ((long) 1) << 32;
+
+    protected List<String> internalGetPropertyNamespaces(String property) {
+        validateAdminAccessOnProperty(property);
+
+        try {
+            return getListOfNamespaces(property);
+        } catch (KeeperException.NoNodeException e) {
+            log.warn("[{}] Failed to get namespace list for propery: {} - Does not exist", clientAppId(), property);
+            throw new RestException(Status.NOT_FOUND, "Property does not exist");
+        } catch (Exception e) {
+            log.error("[{}] Failed to get namespaces list: {}", clientAppId(), e);
+            throw new RestException(e);
+        }
+    }
+
+    protected void internalCreateNamespace(Policies policies) {
+        validatePoliciesReadOnlyAccess();
+        validateAdminAccessOnProperty(namespaceName.getProperty());
+
+        validatePolicies(namespaceName, policies);
+
+        try {
+            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
+
+            zkCreateOptimistic(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies));
+            log.info("[{}] Created namespace {}", clientAppId(), namespaceName);
+        } catch (KeeperException.NodeExistsException e) {
+            log.warn("[{}] Failed to create namespace {} - already exists", clientAppId(), namespaceName);
+            throw new RestException(Status.CONFLICT, "Namespace already exists");
+        } catch (Exception e) {
+            log.error("[{}] Failed to create namespace {}", clientAppId(), namespaceName, e);
+            throw new RestException(e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    protected void internalDeleteNamespace(boolean authoritative) {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
+        validatePoliciesReadOnlyAccess();
+
+        // ensure that non-global namespace is directed to the correct cluster
+        if (!namespaceName.isGlobal()) {
+            validateClusterOwnership(namespaceName.getCluster());
+        }
+
+        Entry<Policies, Stat> 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<String> 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<AuthAction> actions) {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
+
+        try {
+            pulsar().getBrokerService().getAuthorizationService()
+                    .grantPermissionAsync(namespaceName, actions, role, null/*additional auth-data json*/)
+                    .get();
+        } catch (InterruptedException e) {
+            log.error("[{}] Failed to get permissions for namespace {}", clientAppId(), namespaceName, e);
+            throw new RestException(e);
+        } catch (ExecutionException e) {
+            if (e.getCause() instanceof IllegalArgumentException) {
+                log.warn("[{}] Failed to set permissions for namespace {}: does not exist", clientAppId(),
+                        namespaceName);
+                throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
+            } else if (e.getCause() instanceof IllegalStateException) {
+                log.warn("[{}] Failed to set permissions for namespace {}: concurrent modification",
+                        clientAppId(), namespaceName);
+                throw new RestException(Status.CONFLICT, "Concurrent modification");
+            } else {
+                log.error("[{}] Failed to get permissions for namespace {}", clientAppId(), namespaceName, e);
+                throw new RestException(e);
+            }
+        }
+    }
+
+    protected void internalRevokePermissionsOnNamespace(String role) {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
+        validatePoliciesReadOnlyAccess();
+
+        try {
+            Stat nodeStat = new Stat();
+            byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
+            Policies policies = jsonMapper().readValue(content, Policies.class);
+            policies.auth_policies.namespace_auth.remove(role);
+
+            // Write back the new policies into zookeeper
+            globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies),
+                    nodeStat.getVersion());
+
+            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
+            log.info("[{}] Successfully revoked access for role {} - namespace {}", clientAppId(), role, namespaceName);
+        } catch (KeeperException.NoNodeException e) {
+            log.warn("[{}] Failed to revoke permissions for namespace {}: does not exist", clientAppId(),
+                    namespaceName);
+            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
+        } catch (KeeperException.BadVersionException e) {
+            log.warn("[{}] Failed to revoke permissions on namespace {}: concurrent modification", clientAppId(),
+                    namespaceName);
+            throw new RestException(Status.CONFLICT, "Concurrent modification");
+        } catch (Exception e) {
+            log.error("[{}] Failed to revoke permissions on namespace {}", clientAppId(), namespaceName, e);
+            throw new RestException(e);
+        }
+    }
+
+    protected List<String> 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<String> clusterIds) {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
+        validatePoliciesReadOnlyAccess();
+
+        Set<String> 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<String> 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<Policies, Stat> 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<Policies, Stat> 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<Policies, Stat> policiesNode = null;
+
+        try {
+            // Force to read the data s.t. the watch to the cache content is setup.
+            policiesNode = policiesCache().getWithStat(path(POLICIES, namespaceName.toString())).orElseThrow(
+                    () -> new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist"));
+            policiesNode.getKey().deduplicationEnabled = enableDeduplication;
+
+            // Write back the new policies into zookeeper
+            globalZk().setData(path(POLICIES, namespaceName.toString()),
+                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
+            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
+
+            log.info("[{}] Successfully {} on namespace {}", clientAppId(),
+                    enableDeduplication ? "enabled" : "disabled", namespaceName);
+        } catch (KeeperException.NoNodeException e) {
+            log.warn("[{}] Failed to modify deplication status for namespace {}: does not exist", clientAppId(),
+                    namespaceName);
+            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
+        } catch (KeeperException.BadVersionException e) {
+            log.warn(
+                    "[{}] Failed to modify deplication status on namespace {} expected policy node version={} : concurrent modification",
+                    clientAppId(), namespaceName, policiesNode.getValue().getVersion());
+
+            throw new RestException(Status.CONFLICT, "Concurrent modification");
+        } catch (Exception e) {
+            log.error("[{}] Failed to modify deplication status on namespace {}", clientAppId(), namespaceName, e);
+            throw new RestException(e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    protected void internalUnloadNamespace() {
+        log.info("[{}] Unloading namespace {}", clientAppId());
+
+        validateSuperUserAccess();
+
+        if (namespaceName.isGlobal()) {
+            // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
+            validateGlobalNamespaceOwnership(namespaceName);
+        } else {
+            validateClusterOwnership(namespaceName.getCluster());
+            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster());
+        }
+
+        Policies policies = getNamespacePolicies(namespaceName);
+
+        List<String> 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<Policies, Stat> 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<Policies, Stat> 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<BacklogQuota.BacklogQuotaType, BacklogQuota> 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<Topic> topicList = pulsar().getBrokerService().getAllTopicsFromNamespaceBundle(nsName.toString(),
+                    nsName.toString() + "/" + bundleRange);
+
+            List<CompletableFuture<Void>> 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<Topic> topicList = pulsar().getBrokerService().getAllTopicsFromNamespaceBundle(nsName.toString(),
+                    nsName.toString() + "/" + bundleRange);
+            List<CompletableFuture<Void>> 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<String> replicationClusters) {
+        try {
+            ClusterData clusterData = clustersCache().get(path("clusters", clusterName)).orElseThrow(
+                    () -> new RestException(Status.PRECONDITION_FAILED, "Invalid replication cluster " + clusterName));
+            Set<String> peerClusters = clusterData.getPeerClusterNames();
+            if (peerClusters != null && !peerClusters.isEmpty()) {
+                SetView<String> 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<String> partitions = new TreeSet<String>();
+        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<String> bundles = Lists.newArrayList();
+        bundles.addAll(partitions);
+        return new BundlesData(bundles);
+    }
+
+    protected BundlesData getBundles(int numBundles) {
+        if (numBundles <= 0 || numBundles > MAX_BUNDLES) {
+            throw new RestException(Status.BAD_REQUEST,
+                    "Invalid number of bundles. Number of numbles has to be in the range of (0, 2^32].");
+        }
+        Long maxVal = ((long) 1) << 32;
+        Long segSize = maxVal / numBundles;
+        List<String> partitions = Lists.newArrayList();
+        partitions.add(String.format("0x%08x", 0l));
+        Long curPartition = segSize;
+        for (int i = 0; i < numBundles; i++) {
+            if (i != numBundles - 1) {
+                partitions.add(String.format("0x%08x", curPartition));
+            } else {
+                partitions.add(String.format("0x%08x", maxVal - 1));
+            }
+            curPartition += segSize;
+        }
+        return new BundlesData(partitions);
+    }
+
+    private void validatePolicies(NamespaceName ns, Policies policies) {
+        // Validate cluster names and permissions
+        policies.replication_clusters.forEach(cluster -> validateClusterForProperty(ns.getProperty(), cluster));
+
+        if (policies.message_ttl_in_seconds < 0) {
+            throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL");
+        }
+
+        if (policies.bundles != null && policies.bundles.getNumBundles() > 0) {
+            if (policies.bundles.getBoundaries() == null || policies.bundles.getBoundaries().size() == 0) {
+                policies.bundles = getBundles(policies.bundles.getNumBundles());
+            } else {
+                policies.bundles = validateBundlesData(policies.bundles);
+            }
+        } else {
+            int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles();
+            policies.bundles = getBundles(defaultNumberOfBundles);
+        }
+
+        if (policies.persistence != null) {
+            validatePersistencePolicies(policies.persistence);
+        }
+    }
+
+    private static final Logger log = LoggerFactory.getLogger(NamespacesBase.class);
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
similarity index 57%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index 484a87358..b459011d0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+package org.apache.pulsar.broker.admin.impl;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
@@ -35,20 +35,8 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.Encoded;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.container.AsyncResponse;
-import javax.ws.rs.container.Suspended;
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
@@ -65,6 +53,7 @@
 import org.apache.commons.lang3.StringUtils;
 import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException;
 import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
@@ -90,7 +79,6 @@
 import org.apache.pulsar.common.compression.CompressionCodecProvider;
 import org.apache.pulsar.common.naming.DestinationDomain;
 import org.apache.pulsar.common.naming.DestinationName;
-import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.AuthPolicies;
@@ -113,60 +101,44 @@
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.PooledByteBufAllocator;
-import io.swagger.annotations.Api;
-import io.swagger.annotations.ApiOperation;
-import io.swagger.annotations.ApiResponse;
-import io.swagger.annotations.ApiResponses;
 
 /**
  */
-@Path("/persistent")
-@Produces(MediaType.APPLICATION_JSON)
-@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
-public class PersistentTopics extends AdminResource {
-    private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class);
+public class PersistentTopicsBase extends AdminResource {
+    private static final Logger log = LoggerFactory.getLogger(PersistentTopicsBase.class);
 
     protected static final int PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS = 1000;
     private static final int OFFLINE_TOPIC_STAT_TTL_MINS = 10;
     private static final String DEPRECATED_CLIENT_VERSION_PREFIX = "Pulsar-CPP-v";
-    private static final Version LEAST_SUPPORTED_CLIENT_VERSION_PREFIX = Version.forIntegers(1,21);
+    private static final Version LEAST_SUPPORTED_CLIENT_VERSION_PREFIX = Version.forIntegers(1, 21);
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}")
-    @ApiOperation(value = "Get the list of destinations under a namespace.", response = String.class, responseContainer = "List")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
-    public List<String> getList(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
+    protected List<String> internalGetList() {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
 
         // Validate that namespace exists, throws 404 if it doesn't exist
         try {
-            policiesCache().get(path(POLICIES, property, cluster, namespace));
+            policiesCache().get(path(POLICIES, namespaceName.toString()));
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to get topic list {}/{}/{}: Namespace does not exist", clientAppId(), property,
-                    cluster, namespace);
+            log.warn("[{}] Failed to get topic list {}: Namespace does not exist", clientAppId(), namespaceName);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to get topic list {}/{}/{}", clientAppId(), property, cluster, namespace, e);
+            log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
 
         List<String> destinations = Lists.newArrayList();
 
         try {
-            String path = String.format("/managed-ledgers/%s/%s/%s/%s", property, cluster, namespace, domain());
+            String path = String.format("/managed-ledgers/%s/%s", namespaceName.toString(), domain());
             for (String destination : managedLedgerListCache().get(path)) {
                 if (domain().equals(DestinationDomain.persistent.toString())) {
-                    destinations.add(DestinationName
-                            .get(domain(), property, cluster, namespace, decode(destination)).toString());
+                    destinations.add(DestinationName.get(domain(), namespaceName, decode(destination)).toString());
                 }
             }
         } catch (KeeperException.NoNodeException e) {
             // NoNode means there are no destination in this domain for this namespace
         } catch (Exception e) {
-            log.error("[{}] Failed to get destination list for namespace {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
+            log.error("[{}] Failed to get destination list for namespace {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
 
@@ -174,39 +146,34 @@
         return destinations;
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/partitioned")
-    @ApiOperation(value = "Get the list of partitioned topics under a namespace.", response = String.class, responseContainer = "List")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
-    public List<String> getPartitionedTopicList(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
-        validateAdminAccessOnProperty(property);
+    protected List<String> internalGetPartitionedTopicList() {
+        validateAdminAccessOnProperty(namespaceName.getProperty());
 
         // Validate that namespace exists, throws 404 if it doesn't exist
         try {
-            policiesCache().get(path(POLICIES, property, cluster, namespace));
+            policiesCache().get(path(POLICIES, namespaceName.toString()));
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to get partitioned topic list {}/{}/{}: Namespace does not exist", clientAppId(), property,
-                    cluster, namespace);
+            log.warn("[{}] Failed to get partitioned topic list {}: Namespace does not exist", clientAppId(),
+                    namespaceName);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to get partitioned topic list for namespace {}/{}/{}", clientAppId(), property, cluster, namespace, e);
+            log.error("[{}] Failed to get partitioned topic list for namespace {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
 
         List<String> partitionedTopics = Lists.newArrayList();
 
         try {
-            String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain());
+            String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain());
             List<String> destinations = globalZk().getChildren(partitionedTopicPath, false);
-            partitionedTopics = destinations.stream().map(s -> String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, decode(s))).collect(
-                    Collectors.toList());
+            partitionedTopics = destinations.stream()
+                    .map(s -> String.format("persistent://%s/%s", namespaceName.toString(), decode(s)))
+                    .collect(Collectors.toList());
         } catch (KeeperException.NoNodeException e) {
             // NoNode means there are no partitioned topics in this domain for this namespace
         } catch (Exception e) {
-            log.error("[{}] Failed to get partitioned topic list for namespace {}/{}/{}", clientAppId(), property, cluster,
-                    namespace, e);
+            log.error("[{}] Failed to get partitioned topic list for namespace {}", clientAppId(),
+                    namespaceName.toString(), e);
             throw new RestException(e);
         }
 
@@ -214,23 +181,14 @@
         return partitionedTopics;
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions")
-    @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
-            + "namespace level combined (union) with any eventual specific permission set on the destination.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
-    public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination) {
+    protected Map<String, Set<AuthAction>> internalGetPermissionsOnDestination() {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
-        destination = decode(destination);
-        validateAdminAccessOnProperty(property);
+        validateAdminAccessOnProperty(namespaceName.getProperty());
 
-        String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString();
+        String destinationUri = destinationName.toString();
 
         try {
-            Policies policies = policiesCache().get(path(POLICIES, property, cluster, namespace))
+            Policies policies = policiesCache().get(path(POLICIES, namespaceName.toString()))
                     .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist"));
 
             Map<String, Set<AuthAction>> permissions = Maps.newTreeMap();
@@ -264,47 +222,38 @@
         }
     }
 
-    protected void validateAdminAndClientPermission(DestinationName destination) {
+    protected void validateAdminAndClientPermission() {
         try {
-            validateAdminAccessOnProperty(destination.getProperty());
+            validateAdminAccessOnProperty(destinationName.getProperty());
         } catch (Exception ve) {
             try {
-                checkAuthorization(pulsar(), destination, clientAppId(), clientAuthData());
+                checkAuthorization(pulsar(), destinationName, clientAppId(), clientAuthData());
             } catch (RestException re) {
                 throw re;
             } catch (Exception e) {
                 // unknown error marked as internal server error
-                log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}", destination,
-                        clientAppId(), e.getMessage(), e);
+                log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}",
+                        destinationName, clientAppId(), e.getMessage(), e);
                 throw new RestException(e);
             }
         }
     }
 
-    protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
-        validateAdminAccessOnProperty(fqdn.getProperty());
-        validateDestinationOwnership(fqdn, authoritative);
+    public void validateAdminOperationOnDestination(boolean authoritative) {
+        validateAdminAccessOnProperty(destinationName.getProperty());
+        validateDestinationOwnership(destinationName, authoritative);
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
-    @ApiOperation(value = "Grant a new permission to a role on a single destination.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace doesn't exist"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void grantPermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination, @PathParam("role") String role, Set<AuthAction> actions) {
-        destination = decode(destination);
+    protected void internalGrantPermissionsOnDestination(String role, Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
-        validateAdminAccessOnProperty(property);
+        validateAdminAccessOnProperty(namespaceName.getProperty());
         validatePoliciesReadOnlyAccess();
 
-        String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString();
+        String destinationUri = destinationName.toString();
 
         try {
             Stat nodeStat = new Stat();
-            byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat);
+            byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
             Policies policies = jsonMapper().readValue(content, Policies.class);
 
             if (!policies.auth_policies.destination_auth.containsKey(destinationUri)) {
@@ -314,11 +263,11 @@ public void grantPermissionsOnDestination(@PathParam("property") String property
             policies.auth_policies.destination_auth.get(destinationUri).put(role, actions);
 
             // Write the new policies to zookeeper
-            globalZk().setData(path(POLICIES, property, cluster, namespace), jsonMapper().writeValueAsBytes(policies),
+            globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies),
                     nodeStat.getVersion());
 
             // invalidate the local cache to force update
-            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
+            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
 
             log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions,
                     destinationUri);
@@ -333,27 +282,17 @@ public void grantPermissionsOnDestination(@PathParam("property") String property
         }
     }
 
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
-    @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
-            + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
-    public void revokePermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination, @PathParam("role") String role) {
-        destination = decode(destination);
+    protected void internalRevokePermissionsOnDestination(String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
-        validateAdminAccessOnProperty(property);
+        validateAdminAccessOnProperty(namespaceName.getProperty());
         validatePoliciesReadOnlyAccess();
 
-        String destinationUri = DestinationName.get(domain(), property, cluster, namespace, destination).toString();
+        String destinationUri = destinationName.toString();
         Stat nodeStat = new Stat();
         Policies policies;
 
         try {
-            byte[] content = globalZk().getData(path(POLICIES, property, cluster, namespace), null, nodeStat);
+            byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
             policies = jsonMapper().readValue(content, Policies.class);
         } catch (KeeperException.NoNodeException e) {
             log.warn("[{}] Failed to revoke permissions on destination {}: Namespace does not exist", clientAppId(),
@@ -375,7 +314,7 @@ public void revokePermissionsOnDestination(@PathParam("property") String propert
 
         try {
             // Write the new policies to zookeeper
-            String namespacePath = path(POLICIES, property, cluster, namespace);
+            String namespacePath = path(POLICIES, namespaceName.toString());
             globalZk().setData(namespacePath, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
 
             // invalidate the local cache to force update
@@ -390,33 +329,24 @@ public void revokePermissionsOnDestination(@PathParam("property") String propert
         }
     }
 
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 409, message = "Partitioned topic already exist") })
-    public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAccessOnProperty(dn.getProperty());
+    protected void internalCreatePartitionedTopic(int numPartitions, boolean authoritative) {
+        validateAdminAccessOnProperty(destinationName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
-            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(),
-                    dn.getEncodedLocalName());
+            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
+                    destinationName.getEncodedLocalName());
             byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
             zkCreateOptimistic(path, data);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), dn);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
         } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), dn);
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
             throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), dn, e);
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
             throw new RestException(e);
         }
     }
@@ -430,77 +360,47 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path
      * recreate them at application so, newly created producers and consumers can connect to newly added partitions as
      * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application.
      *
-     * @param property
-     * @param cluster
-     * @param namespace
-     * @param destination
      * @param numPartitions
      */
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 409, message = "Partitioned topic does not exist") })
-    public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            int numPartitions) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAccessOnProperty(dn.getProperty());
-        if (dn.isGlobal()) {
-            log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(), dn);
+    protected void internalUpdatePartitionedTopic(int numPartitions) {
+        validateAdminAccessOnProperty(destinationName.getProperty());
+        if (destinationName.isGlobal()) {
+            log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(),
+                    destinationName);
             throw new RestException(Status.FORBIDDEN, "Update forbidden on global namespace");
         }
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
-            updatePartitionedTopic(dn, numPartitions).get();
+            updatePartitionedTopic(destinationName, numPartitions).get();
         } catch (Exception e) {
             if (e.getCause() instanceof RestException) {
                 throw (RestException) e.getCause();
             }
-            log.error("[{}] Failed to update partitioned topic {}", clientAppId(), dn, e.getCause());
+            log.error("[{}] Failed to update partitioned topic {}", clientAppId(), destinationName, e.getCause());
             throw new RestException(e.getCause());
         }
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Get partitioned topic metadata.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
-    public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative);
+    protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean authoritative) {
+        PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (metadata.partitions > 1) {
             validateClientVersion();
         }
         return metadata;
     }
 
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Partitioned topic does not exist") })
-    public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAccessOnProperty(dn.getProperty());
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalDeletePartitionedTopic(boolean authoritative) {
+        validateAdminAccessOnProperty(destinationName.getProperty());
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         int numPartitions = partitionMetadata.partitions;
         if (numPartitions > 0) {
             final CompletableFuture<Void> future = new CompletableFuture<>();
             final AtomicInteger count = new AtomicInteger(numPartitions);
             try {
                 for (int i = 0; i < numPartitions; i++) {
-                    DestinationName dn_partition = dn.getPartition(i);
+                    DestinationName dn_partition = destinationName.getPartition(i);
                     pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString())
                             .whenComplete((r, ex) -> {
                                 if (ex != null) {
@@ -538,63 +438,45 @@ public void deletePartitionedTopic(@PathParam("property") String property, @Path
         }
 
         // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted
-        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(),
-                dn.getEncodedLocalName());
+        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
+                destinationName.getEncodedLocalName());
         try {
             globalZk().delete(path, -1);
             globalZkCache().invalidate(path);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Deleted partitioned topic {}", clientAppId(), dn);
+            log.info("[{}] Deleted partitioned topic {}", clientAppId(), destinationName);
         } catch (KeeperException.NoNodeException nne) {
             throw new RestException(Status.NOT_FOUND, "Partitioned topic does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), dn, e);
+            log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), destinationName, e);
             throw new RestException(e);
         }
     }
 
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
-    @ApiOperation(value = "Unload a topic")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        log.info("[{}] Unloading topic {}/{}/{}/{}", clientAppId(), property, cluster, namespace, destination);
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        unloadTopic(dn, authoritative);
+    protected void internalUnloadTopic(boolean authoritative) {
+        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        unloadTopic(destinationName, authoritative);
     }
 
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}")
-    @ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist"),
-            @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
-    public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminOperationOnDestination(dn, authoritative);
-        Topic topic = getTopicReference(dn);
-        if (dn.isGlobal()) {
+    protected void internalDeleteTopic(boolean authoritative) {
+        validateAdminOperationOnDestination(authoritative);
+        Topic topic = getTopicReference(destinationName);
+        if (destinationName.isGlobal()) {
             // Delete is disallowed on global topic
-            log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), dn);
+            log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), destinationName);
             throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace");
         }
+
         try {
             topic.delete().get();
-            log.info("[{}] Successfully removed topic {}", clientAppId(), dn);
+            log.info("[{}] Successfully removed topic {}", clientAppId(), destinationName);
         } catch (Exception e) {
             Throwable t = e.getCause();
-            log.error("[{}] Failed to get delete topic {}", clientAppId(), dn, t);
+            log.error("[{}] Failed to get delete topic {}", clientAppId(), destinationName, t);
             if (t instanceof TopicBusyException) {
                 throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions");
             } else {
@@ -603,39 +485,30 @@ public void deleteTopic(@PathParam("property") String property, @PathParam("clus
         }
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions")
-    @ApiOperation(value = "Get the list of persistent subscriptions for a given topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public List<String> getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
+    protected List<String> internalGetSubscriptions(boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
         }
+
         List<String> subscriptions = Lists.newArrayList();
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 // get the subscriptions only from the 1st partition since all the other partitions will have the same
                 // subscriptions
-                subscriptions.addAll(
-                        pulsar().getAdminClient().persistentTopics().getSubscriptions(dn.getPartition(0).toString()));
+                subscriptions.addAll(pulsar().getAdminClient().persistentTopics()
+                        .getSubscriptions(destinationName.getPartition(0).toString()));
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
-            validateAdminOperationOnDestination(dn, authoritative);
-            Topic topic = getTopicReference(dn);
+            validateAdminOperationOnDestination(authoritative);
+            Topic topic = getTopicReference(destinationName);
 
             try {
                 topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName));
             } catch (Exception e) {
-                log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), dn);
+                log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), destinationName);
                 throw new RestException(e);
             }
         }
@@ -643,60 +516,32 @@ public void deleteTopic(@PathParam("property") String property, @PathParam("clus
         return subscriptions;
     }
 
-    @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/stats")
-    @ApiOperation(value = "Get the stats for the topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAndClientPermission(dn);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        validateDestinationOwnership(dn, authoritative);
-        Topic topic = getTopicReference(dn);
+    protected PersistentTopicStats internalGetStats(boolean authoritative) {
+        validateAdminAndClientPermission();
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        validateDestinationOwnership(destinationName, authoritative);
+        Topic topic = getTopicReference(destinationName);
         return topic.getStats();
     }
 
-    @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
-    @ApiOperation(value = "Get the internal stats for the topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAndClientPermission(dn);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        validateDestinationOwnership(dn, authoritative);
-        Topic topic = getTopicReference(dn);
+    protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) {
+        validateAdminAndClientPermission();
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        validateDestinationOwnership(destinationName, authoritative);
+        Topic topic = getTopicReference(destinationName);
         return topic.getInternalStats();
     }
 
-    @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internal-info")
-    @ApiOperation(value = "Get the internal stats for the topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @Suspended AsyncResponse asyncResponse) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAccessOnProperty(dn.getProperty());
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        String managedLedger = dn.getPersistenceNamingEncoding();
+    protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse) {
+        validateAdminAccessOnProperty(destinationName.getProperty());
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        String managedLedger = destinationName.getPersistenceNamingEncoding();
         pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() {
             @Override
             public void getInfoComplete(ManagedLedgerInfo info, Object ctx) {
@@ -712,32 +557,21 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) {
         }, null);
     }
 
-    @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats")
-    @ApiOperation(value = "Get the stats for the partitioned topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected PartitionedTopicStats internalGetPartitionedStats(boolean authoritative) {
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions == 0) {
             throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found");
         }
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
         }
         PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata);
         try {
             for (int i = 0; i < partitionMetadata.partitions; i++) {
                 PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics()
-                        .getStats(dn.getPartition(i).toString());
+                        .getStats(destinationName.getPartition(i).toString());
                 stats.add(partitionStats);
-                stats.partitions.put(dn.getPartition(i).toString(), partitionStats);
+                stats.partitions.put(destinationName.getPartition(i).toString(), partitionStats);
             }
         } catch (Exception e) {
             throw new RestException(e);
@@ -745,28 +579,16 @@ public PartitionedTopicStats getPartitionedStats(@PathParam("property") String p
         return stats;
     }
 
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}")
-    @ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic does not exist"),
-            @ApiResponse(code = 412, message = "Subscription has active consumers") })
-    public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalDeleteSubscription(String subName, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
-                    pulsar().getAdminClient().persistentTopics().deleteSubscription(dn.getPartition(i).toString(),
-                            subName);
+                    pulsar().getAdminClient().persistentTopics()
+                            .deleteSubscription(destinationName.getPartition(i).toString(), subName);
                 }
             } catch (Exception e) {
                 if (e instanceof NotFoundException) {
@@ -774,18 +596,18 @@ public void deleteSubscription(@PathParam("property") String property, @PathPara
                 } else if (e instanceof PreconditionFailedException) {
                     throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
                 } else {
-                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), dn, subName, e);
+                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
                     throw new RestException(e);
                 }
             }
         } else {
-            validateAdminOperationOnDestination(dn, authoritative);
-            Topic topic = getTopicReference(dn);
+            validateAdminOperationOnDestination(authoritative);
+            Topic topic = getTopicReference(destinationName);
             try {
                 Subscription sub = topic.getSubscription(subName);
                 checkNotNull(sub);
                 sub.delete().get();
-                log.info("[{}][{}] Deleted subscription {}", clientAppId(), dn, subName);
+                log.info("[{}][{}] Deleted subscription {}", clientAppId(), destinationName, subName);
             } catch (Exception e) {
                 Throwable t = e.getCause();
                 if (e instanceof NullPointerException) {
@@ -793,43 +615,30 @@ public void deleteSubscription(@PathParam("property") String property, @PathPara
                 } else if (t instanceof SubscriptionBusyException) {
                     throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
                 } else {
-                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), dn, subName, e);
+                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
                     throw new RestException(t);
                 }
             }
         }
-
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all")
-    @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
-            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
-    public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalSkipAllMessages(String subName, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
-                    pulsar().getAdminClient().persistentTopics().skipAllMessages(dn.getPartition(i).toString(),
-                            subName);
+                    pulsar().getAdminClient().persistentTopics()
+                            .skipAllMessages(destinationName.getPartition(i).toString(), subName);
                 }
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
-            validateAdminOperationOnDestination(dn, authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+            validateAdminOperationOnDestination(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
             try {
                 if (subName.startsWith(topic.replicatorPrefix)) {
                     String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -841,38 +650,26 @@ public void skipAllMessages(@PathParam("property") String property, @PathParam("
                     checkNotNull(sub);
                     sub.clearBacklog().get();
                 }
-                log.info("[{}] Cleared backlog on {} {}", clientAppId(), dn, subName);
+                log.info("[{}] Cleared backlog on {} {}", clientAppId(), destinationName, subName);
             } catch (NullPointerException npe) {
                 throw new RestException(Status.NOT_FOUND, "Subscription not found");
             } catch (Exception exception) {
-                log.error("[{}] Failed to skip all messages {} {}", clientAppId(), dn, subName, exception);
+                log.error("[{}] Failed to skip all messages {} {}", clientAppId(), destinationName, subName, exception);
                 throw new RestException(exception);
             }
         }
-
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
-    @ApiOperation(value = "Skip messages on a topic subscription.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
-    public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName, @PathParam("numMessages") int numMessages,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalSkipMessages(String subName, int numMessages, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Skip messages on a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(dn, authoritative);
-        PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+        validateAdminOperationOnDestination(authoritative);
+        PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
         try {
             if (subName.startsWith(topic.replicatorPrefix)) {
                 String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -884,85 +681,51 @@ public void skipMessages(@PathParam("property") String property, @PathParam("clu
                 checkNotNull(sub);
                 sub.skipMessages(numMessages).get();
             }
-            log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, dn, subName);
+            log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, destinationName, subName);
         } catch (NullPointerException npe) {
             throw new RestException(Status.NOT_FOUND, "Subscription not found");
         } catch (Exception exception) {
-            log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, dn, subName, exception);
+            log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, destinationName, subName,
+                    exception);
             throw new RestException(exception);
         }
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
-    @ApiOperation(value = "Expire messages on a topic subscription.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
-    public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative);
-    }
-
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
-    @ApiOperation(value = "Expire messages on all subscriptions of topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
-    public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destinationName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        final String destination = decode(destinationName);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalExpireMessagesForAllSubscriptions(int expireTimeInSeconds, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 // expire messages for each partition destination
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
-                    pulsar().getAdminClient().persistentTopics()
-                            .expireMessagesForAllSubscriptions(dn.getPartition(i).toString(), expireTimeInSeconds);
+                    pulsar().getAdminClient().persistentTopics().expireMessagesForAllSubscriptions(
+                            destinationName.getPartition(i).toString(), expireTimeInSeconds);
                 }
             } catch (Exception e) {
-                log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn,
-                        e);
+                log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds,
+                        destinationName, e);
                 throw new RestException(e);
             }
         } else {
             // validate ownership and redirect if current broker is not owner
-            validateAdminOperationOnDestination(dn, authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+            validateAdminOperationOnDestination(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
             topic.getReplicators().forEach((subName, replicator) -> {
-                expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative);
+                internalExpireMessages(subName, expireTimeInSeconds, authoritative);
             });
             topic.getSubscriptions().forEach((subName, subscriber) -> {
-                expireMessages(property, cluster, namespace, destination, subName, expireTimeInSeconds, authoritative);
+                internalExpireMessages(subName, expireTimeInSeconds, authoritative);
             });
         }
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
-    @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
-    public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName, @PathParam("timestamp") long timestamp,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected void internalResetCursor(String subName, long timestamp, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
 
         if (partitionMetadata.partitions > 0) {
             int numParts = partitionMetadata.partitions;
@@ -970,8 +733,8 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus
             Exception partitionException = null;
             try {
                 for (int i = 0; i < numParts; i++) {
-                    pulsar().getAdminClient().persistentTopics().resetCursor(dn.getPartition(i).toString(), subName,
-                            timestamp);
+                    pulsar().getAdminClient().persistentTopics().resetCursor(destinationName.getPartition(i).toString(),
+                            subName, timestamp);
                 }
             } catch (PreconditionFailedException pfe) {
                 // throw the last exception if all partitions get this error
@@ -979,25 +742,25 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus
                 ++numPartException;
                 partitionException = pfe;
             } catch (Exception e) {
-                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName,
-                        timestamp, e);
+                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
+                        destinationName, subName, timestamp, e);
                 throw new RestException(e);
             }
             // report an error to user if unable to reset for all partitions
             if (numPartException == numParts) {
-                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName,
-                        timestamp, partitionException);
+                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
+                        destinationName, subName, timestamp, partitionException);
                 throw new RestException(Status.PRECONDITION_FAILED, partitionException.getMessage());
             } else if (numPartException > 0) {
                 log.warn("[{}][{}] partial errors for reset cursor on subscription {} to time {} - ", clientAppId(),
-                        destination, subName, timestamp, partitionException);
+                        destinationName, subName, timestamp, partitionException);
             }
 
         } else {
-            validateAdminOperationOnDestination(dn, authoritative);
-            log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destination,
+            validateAdminOperationOnDestination(authoritative);
+            log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destinationName,
                     subName, timestamp);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
             if (topic == null) {
                 throw new RestException(Status.NOT_FOUND, "Topic not found");
             }
@@ -1005,11 +768,12 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus
                 PersistentSubscription sub = topic.getSubscription(subName);
                 checkNotNull(sub);
                 sub.resetCursor(timestamp).get();
-                log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), dn, subName, timestamp);
+                log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), destinationName, subName,
+                        timestamp);
             } catch (Exception e) {
                 Throwable t = e.getCause();
-                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), dn, subName,
-                        timestamp, e);
+                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
+                        destinationName, subName, timestamp, e);
                 if (e instanceof NullPointerException) {
                     throw new RestException(Status.NOT_FOUND, "Subscription not found");
                 } else if (e instanceof NotAllowedException) {
@@ -1024,26 +788,14 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus
         }
     }
 
-    @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}")
-    @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
-            @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
-    public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subscriptionName") String subscriptionName,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) throws PulsarServerException {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destination,
+    protected void internalCreateSubscription(String subscriptionName, MessageIdImpl messageId, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destinationName,
                 subscriptionName, messageId);
 
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
 
         try {
             if (partitionMetadata.partitions > 0) {
@@ -1052,15 +804,16 @@ public void createSubscription(@PathParam("property") String property, @PathPara
                 PulsarAdmin admin = pulsar().getAdminClient();
 
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
-                    futures.add(admin.persistentTopics().createSubscriptionAsync(dn.getPartition(i).toString(),
+                    futures.add(admin.persistentTopics().createSubscriptionAsync(
+                            destinationName.getPartition(i).toString(),
                             subscriptionName, messageId));
                 }
 
                 FutureUtil.waitForAll(futures).join();
             } else {
-                validateAdminOperationOnDestination(dn, authoritative);
+                validateAdminOperationOnDestination(authoritative);
 
-                PersistentTopic topic = (PersistentTopic) getOrCreateTopic(dn);
+                PersistentTopic topic = (PersistentTopic) getOrCreateTopic(destinationName);
 
                 if (topic.getSubscriptions().containsKey(subscriptionName)) {
                     throw new RestException(Status.CONFLICT, "Subscription already exists for topic");
@@ -1069,13 +822,13 @@ public void createSubscription(@PathParam("property") String property, @PathPara
                 PersistentSubscription subscription = (PersistentSubscription) topic
                         .createSubscription(subscriptionName).get();
                 subscription.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
-                log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), dn,
-                        subscriptionName, messageId);
+                log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(),
+                        destinationName, subscriptionName, messageId);
             }
         } catch (Exception e) {
             Throwable t = e.getCause();
-            log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), dn, subscriptionName,
-                    messageId, e);
+            log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(),
+                    destinationName, subscriptionName, messageId, e);
             if (t instanceof SubscriptionInvalidCursorPosition) {
                 throw new RestException(Status.PRECONDITION_FAILED,
                         "Unable to find position for position specified: " + t.getMessage());
@@ -1085,34 +838,23 @@ public void createSubscription(@PathParam("property") String property, @PathPara
         }
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor")
-    @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
-            @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
-    public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destination,
+    protected void internalResetCursorOnPosition(String subName, boolean authoritative, MessageIdImpl messageId) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destinationName,
                 subName, messageId);
 
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
 
         if (partitionMetadata.partitions > 0) {
-            log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), dn, subName);
+            log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), destinationName,
+                    subName);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
                     "Reset-cursor at position is not allowed for partitioned-topic");
         } else {
-            validateAdminOperationOnDestination(dn, authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+            validateAdminOperationOnDestination(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
             if (topic == null) {
                 throw new RestException(Status.NOT_FOUND, "Topic not found");
             }
@@ -1120,12 +862,12 @@ public void resetCursorOnPosition(@PathParam("property") String property, @PathP
                 PersistentSubscription sub = topic.getSubscription(subName);
                 checkNotNull(sub);
                 sub.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
-                log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(), dn,
-                        subName, messageId);
+                log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(),
+                        destinationName, subName, messageId);
             } catch (Exception e) {
                 Throwable t = e.getCause();
-                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), dn,
-                        subName, messageId, e);
+                log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(),
+                        destinationName, subName, messageId, e);
                 if (e instanceof NullPointerException) {
                     throw new RestException(Status.NOT_FOUND, "Subscription not found");
                 } else if (t instanceof SubscriptionInvalidCursorPosition) {
@@ -1138,32 +880,22 @@ public void resetCursorOnPosition(@PathParam("property") String property, @PathP
         }
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
-    @ApiOperation(value = "Peek nth message on a topic subscription.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
-    public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected Response internalPeekNthMessage(String subName, int messagePosition, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Peek messages on a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(dn, authoritative);
-        if (!(getTopicReference(dn) instanceof PersistentTopic)) {
-            log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), dn, subName);
+        validateAdminOperationOnDestination(authoritative);
+        if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
+            log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+                    subName);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
                     "Skip messages on a non-persistent topic is not allowed");
         }
-        PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+        PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
         PersistentReplicator repl = null;
         PersistentSubscription sub = null;
         Entry entry = null;
@@ -1223,8 +955,8 @@ public void write(OutputStream output) throws IOException, WebApplicationExcepti
         } catch (NullPointerException npe) {
             throw new RestException(Status.NOT_FOUND, "Message not found");
         } catch (Exception exception) {
-            log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition, dn,
-                    subName, exception);
+            log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition,
+                    destinationName, subName, exception);
             throw new RestException(exception);
         } finally {
             if (entry != null) {
@@ -1233,37 +965,26 @@ public void write(OutputStream output) throws IOException, WebApplicationExcepti
         }
     }
 
-    @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/backlog")
-    @ApiOperation(value = "Get estimated backlog for offline topic.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        validateAdminAccessOnProperty(property);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
+    protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
         }
         // Validate that namespace exists, throw 404 if it doesn't exist
         // note that we do not want to load the topic and hence skip validateAdminOperationOnDestination()
         try {
-            policiesCache().get(path(POLICIES, property, cluster, namespace));
+            policiesCache().get(path(POLICIES, namespaceName.toString()));
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to get topic backlog {}/{}/{}: Namespace does not exist", clientAppId(), property,
-                    cluster, namespace);
+            log.warn("[{}] Failed to get topic backlog {}: Namespace does not exist", clientAppId(), namespaceName);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to get topic backlog {}/{}/{}", clientAppId(), property, cluster, namespace, e);
+            log.error("[{}] Failed to get topic backlog {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
+
         PersistentOfflineTopicStats offlineTopicStats = null;
         try {
 
-            offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(dn);
+            offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(destinationName);
             if (offlineTopicStats != null) {
                 // offline topic stat has a cost - so use cached value until TTL
                 long elapsedMs = System.currentTimeMillis() - offlineTopicStats.statGeneratedAt.getTime();
@@ -1271,75 +992,62 @@ public PersistentOfflineTopicStats getBacklog(@PathParam("property") String prop
                     return offlineTopicStats;
                 }
             }
-            final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(dn).get();
+            final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(destinationName)
+                    .get();
             ManagedLedgerOfflineBacklog offlineTopicBacklog = new ManagedLedgerOfflineBacklog(config.getDigestType(),
                     config.getPassword(), pulsar().getAdvertisedAddress(), false);
-            offlineTopicStats = offlineTopicBacklog
-                    .estimateUnloadedTopicBacklog((ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), dn);
-            pulsar().getBrokerService().cacheOfflineTopicStats(dn, offlineTopicStats);
+            offlineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog(
+                    (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), destinationName);
+            pulsar().getBrokerService().cacheOfflineTopicStats(destinationName, offlineTopicStats);
         } catch (Exception exception) {
             throw new RestException(exception);
         }
         return offlineTopicStats;
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/terminate")
-    @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more "
-            + "messages to be published and will let consumer to drain existing messages in backlog")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
-            @ApiResponse(code = 404, message = "Topic does not exist") })
-    public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
-        }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+    protected MessageId internalTerminate(boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(dn, authoritative);
-        Topic topic = getTopicReference(dn);
+        validateAdminOperationOnDestination(authoritative);
+        Topic topic = getTopicReference(destinationName);
         try {
             return ((PersistentTopic) topic).terminate().get();
         } catch (Exception exception) {
-            log.error("[{}] Failed to terminated topic {}", clientAppId(), dn, exception);
+            log.error("[{}] Failed to terminated topic {}", clientAppId(), destinationName, exception);
             throw new RestException(exception);
         }
     }
 
-    public void expireMessages(String property, String cluster, String namespace, String destination, String subName,
-            int expireTimeInSeconds, boolean authoritative) {
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
+    protected void internalExpireMessages(String subName, int expireTimeInSeconds, boolean authoritative) {
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
-                destination, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
         if (partitionMetadata.partitions > 0) {
             // expire messages for each partition destination
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
-                    pulsar().getAdminClient().persistentTopics().expireMessages(dn.getPartition(i).toString(), subName,
-                            expireTimeInSeconds);
+                    pulsar().getAdminClient().persistentTopics()
+                            .expireMessages(destinationName.getPartition(i).toString(), subName, expireTimeInSeconds);
                 }
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
             // validate ownership and redirect if current broker is not owner
-            validateAdminOperationOnDestination(dn, authoritative);
-            if (!(getTopicReference(dn) instanceof PersistentTopic)) {
-                log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), dn, subName);
+            validateAdminOperationOnDestination(authoritative);
+            if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
+                log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+                        subName);
                 throw new RestException(Status.METHOD_NOT_ALLOWED,
                         "Expire messages on a non-persistent topic is not allowed");
             }
-            PersistentTopic topic = (PersistentTopic) getTopicReference(dn);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
             try {
                 if (subName.startsWith(topic.replicatorPrefix)) {
                     String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -1351,13 +1059,13 @@ public void expireMessages(String property, String cluster, String namespace, St
                     checkNotNull(sub);
                     sub.expireMessages(expireTimeInSeconds);
                 }
-                log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds, dn,
-                        subName);
+                log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds,
+                        destinationName, subName);
             } catch (NullPointerException npe) {
                 throw new RestException(Status.NOT_FOUND, "Subscription not found");
             } catch (Exception exception) {
                 log.error("[{}] Failed to expire messages up to {} on {} with subscription {} {}", clientAppId(),
-                        expireTimeInSeconds, dn, subName, exception);
+                        expireTimeInSeconds, destinationName, subName, exception);
                 throw new RestException(exception);
             }
         }
@@ -1386,8 +1094,8 @@ public void expireMessages(String property, String cluster, String namespace, St
                 throw ex;
             }
 
-            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(),
-                    dn.getNamespacePortion(), "persistent", dn.getEncodedLocalName());
+            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getNamespace(),
+                    "persistent", dn.getEncodedLocalName());
 
             // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
             // serve/redirect request else fail partitioned-metadata-request so, client fails while creating
@@ -1409,7 +1117,7 @@ public void expireMessages(String property, String cluster, String namespace, St
         return metadataFuture;
     }
 
-	/**
+    /**
      * Get the Topic object reference from the Pulsar broker
      */
     private Topic getTopicReference(DestinationName dn) {
@@ -1485,8 +1193,10 @@ private PersistentReplicator getReplicatorReference(String replName, PersistentT
     /**
      * It creates subscriptions for new partitions of existing partitioned-topics
      *
-     * @param dn : topic-name: persistent://prop/cluster/ns/topic
-     * @param numPartitions : number partitions for the topics
+     * @param dn
+     *            : topic-name: persistent://prop/cluster/ns/topic
+     * @param numPartitions
+     *            : number partitions for the topics
      */
     private CompletableFuture<Void> createSubscriptions(DestinationName dn, int numPartitions) {
         String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(),
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
similarity index 95%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
index 359f3002c..6845e91b4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Properties.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
@@ -16,22 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+package org.apache.pulsar.broker.admin.impl;
 
-import java.util.Collections;
 import java.util.List;
 
-import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
+import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.naming.NamedEntity;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -41,17 +38,12 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
-import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
 
-@Path("/properties")
-@Produces(MediaType.APPLICATION_JSON)
-@Consumes(MediaType.APPLICATION_JSON)
-@Api(value = "/properties", description = "Properties admin apis", tags = "properties")
-public class Properties extends AdminResource {
+public class PropertiesBase extends AdminResource {
 
     @GET
     @ApiOperation(value = "Get the list of properties.", response = String.class, responseContainer = "List")
@@ -207,5 +199,5 @@ public void deleteProperty(@PathParam("property") String property) {
         }
     }
 
-    private static final Logger log = LoggerFactory.getLogger(Properties.class);
+    private static final Logger log = LoggerFactory.getLogger(PropertiesBase.class);
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java
similarity index 53%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java
index 3eed02328..9c5a54f8c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ResourceQuotasBase.java
@@ -16,43 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
-
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
+package org.apache.pulsar.broker.admin.impl;
+
 import javax.ws.rs.core.Response.Status;
 
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.ResourceQuota;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.swagger.annotations.Api;
-import io.swagger.annotations.ApiOperation;
-import io.swagger.annotations.ApiResponse;
-import io.swagger.annotations.ApiResponses;
-
-@Path("/resource-quotas")
-@Produces(MediaType.APPLICATION_JSON)
-@Consumes(MediaType.APPLICATION_JSON)
-@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas")
-public class ResourceQuotas extends AdminResource {
+public abstract class ResourceQuotasBase extends NamespacesBase {
 
-    private static final Logger log = LoggerFactory.getLogger(ResourceQuotas.class);
-
-    @GET
-    @ApiOperation(value = "Get the default quota", response = String.class, responseContainer = "Set")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public ResourceQuota getDefaultResourceQuota() throws Exception {
         validateSuperUserAccess();
         try {
@@ -64,9 +41,6 @@ public ResourceQuota getDefaultResourceQuota() throws Exception {
 
     }
 
-    @POST
-    @ApiOperation(value = "Set the default quota", response = String.class, responseContainer = "Set")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public void setDefaultResourceQuota(ResourceQuota quota) throws Exception {
         validateSuperUserAccess();
         validatePoliciesReadOnlyAccess();
@@ -78,25 +52,18 @@ public void setDefaultResourceQuota(ResourceQuota quota) throws Exception {
         }
     }
 
-    @GET
-    @Path("/{property}/{cluster}/{namespace}/{bundle}")
-    @ApiOperation(value = "Get resource quota of a namespace bundle.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 404, message = "Namespace does not exist") })
-    public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("bundle") String bundleRange) {
+    @SuppressWarnings("deprecation")
+    protected ResourceQuota internalGetNamespaceBundleResourceQuota(String bundleRange) {
         validateSuperUserAccess();
 
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
+        Policies policies = getNamespacePolicies(namespaceName);
 
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
+        if (!namespaceName.isGlobal()) {
+            validateClusterOwnership(namespaceName.getCluster());
+            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster());
         }
 
-        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
-        NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
+        NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange);
 
         try {
             return pulsar().getLocalZkCacheService().getResourceQuotaCache().getQuota(nsBundle);
@@ -106,26 +73,19 @@ public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") Stri
         }
     }
 
-    @POST
-    @Path("/{property}/{cluster}/{namespace}/{bundle}")
-    @ApiOperation(value = "Set resource quota on a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void setNamespaceBundleResourceQuota(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("bundle") String bundleRange, ResourceQuota quota) {
+    @SuppressWarnings("deprecation")
+    protected void internalSetNamespaceBundleResourceQuota(String bundleRange, ResourceQuota quota) {
         validateSuperUserAccess();
         validatePoliciesReadOnlyAccess();
 
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
+        Policies policies = getNamespacePolicies(namespaceName);
 
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
+        if (!namespaceName.isGlobal()) {
+            validateClusterOwnership(namespaceName.getCluster());
+            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster());
         }
 
-        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
-        NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
+        NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange);
 
         try {
             pulsar().getLocalZkCacheService().getResourceQuotaCache().setQuota(nsBundle, quota);
@@ -142,26 +102,19 @@ public void setNamespaceBundleResourceQuota(@PathParam("property") String proper
 
     }
 
-    @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{bundle}")
-    @ApiOperation(value = "Remove resource quota for a namespace.")
-    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void removeNamespaceBundleResourceQuota(@PathParam("property") String property,
-            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("bundle") String bundleRange) {
+    @SuppressWarnings("deprecation")
+    protected void internalRemoveNamespaceBundleResourceQuota(String bundleRange) {
         validateSuperUserAccess();
         validatePoliciesReadOnlyAccess();
 
-        Policies policies = getNamespacePolicies(property, cluster, namespace);
+        Policies policies = getNamespacePolicies(namespaceName);
 
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateClusterOwnership(cluster);
-            validateClusterForProperty(property, cluster);
+        if (!namespaceName.isGlobal()) {
+            validateClusterOwnership(namespaceName.getCluster());
+            validateClusterForProperty(namespaceName.getProperty(), namespaceName.getCluster());
         }
 
-        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
-        NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
+        NamespaceBundle nsBundle = validateNamespaceBundleRange(namespaceName, policies.bundles, bundleRange);
 
         try {
             pulsar().getLocalZkCacheService().getResourceQuotaCache().unsetQuota(nsBundle);
@@ -177,4 +130,6 @@ public void removeNamespaceBundleResourceQuota(@PathParam("property") String pro
             throw new RestException(e);
         }
     }
+
+    private static final Logger log = LoggerFactory.getLogger(ResourceQuotasBase.class);
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/BrokerStats.java
new file mode 100644
index 000000000..97ebbd9a7
--- /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 000000000..c5d712cb3
--- /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 000000000..7cd5ccd45
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Clusters.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v1;
+
+import io.swagger.annotations.Api;
+import org.apache.pulsar.broker.admin.impl.ClustersBase;
+
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("/clusters")
+@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters")
+@Produces(MediaType.APPLICATION_JSON)
+public class Clusters extends ClustersBase {
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
new file mode 100644
index 000000000..3ae1d4bca
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
@@ -0,0 +1,711 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v1;
+
+import static org.apache.commons.lang3.StringUtils.isBlank;
+import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
+import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES_ROOT;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.pulsar.broker.admin.AdminResource;
+import org.apache.pulsar.broker.admin.impl.NamespacesBase;
+import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.BacklogQuota;
+import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
+import org.apache.pulsar.common.policies.data.BundlesData;
+import org.apache.pulsar.common.policies.data.DispatchRate;
+import org.apache.pulsar.common.policies.data.PersistencePolicies;
+import org.apache.pulsar.common.policies.data.Policies;
+import org.apache.pulsar.common.policies.data.RetentionPolicies;
+import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+@Path("/namespaces")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
+public class Namespaces extends NamespacesBase {
+
+    @GET
+    @Path("/{property}")
+    @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property doesn't exist") })
+    public List<String> getPropertyNamespaces(@PathParam("property") String property) {
+        return internalGetPropertyNamespaces(property);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}")
+    @ApiOperation(hidden = true, value = "Get the list of all the namespaces for a certain property on single cluster.", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster doesn't exist") })
+    public List<String> getNamespacesForCluster(@PathParam("property") String property,
+            @PathParam("cluster") String cluster) {
+        validateAdminAccessOnProperty(property);
+        List<String> 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<String> 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<String, Set<AuthAction>> 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<AuthAction> 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<String> 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<String> clusterIds) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetNamespaceReplicationClusters(clusterIds);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/messageTTL")
+    @ApiOperation(hidden = true, value = "Get the message TTL for the namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public int getNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, cluster, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+        return policies.message_ttl_in_seconds;
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/messageTTL")
+    @ApiOperation(hidden = true, value = "Set message TTL in seconds for namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Invalid TTL") })
+    public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, int messageTTL) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetNamespaceMessageTTL(messageTTL);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/antiAffinity")
+    @ApiOperation(value = "Set anti-affinity group for a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Invalid antiAffinityGroup") })
+    public void setNamespaceAntiAffinityGroup(@PathParam("property") String property, @PathParam("cluster") String cluster,
+                                              @PathParam("namespace") String namespace, String antiAffinityGroup) {
+        validateAdminAccessOnProperty(property);
+        validatePoliciesReadOnlyAccess();
+
+        log.info("[{}] Setting anti-affinity group {} for {}/{}/{}", clientAppId(), antiAffinityGroup, property,
+                cluster, namespace);
+
+        if (isBlank(antiAffinityGroup)) {
+            throw new RestException(Status.PRECONDITION_FAILED, "antiAffinityGroup can't be empty");
+        }
+
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
+        Map.Entry<Policies, Stat> 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<String> 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<String> namespaces = Lists.newArrayList();
+        try {
+            for (String prop : globalZk().getChildren(POLICIES_ROOT, false)) {
+                for (String namespace : globalZk().getChildren(path(POLICIES, prop, cluster), false)) {
+                    Optional<Policies> policies = policiesCache()
+                            .get(AdminResource.path(POLICIES, prop, cluster, namespace));
+                    if (policies.isPresent() && antiAffinityGroup.equalsIgnoreCase(policies.get().antiAffinityGroup)) {
+                        namespaces.add(String.format("%s/%s/%s", prop, cluster, namespace));
+                    }
+                }
+            }
+        } catch (Exception e) {
+            log.warn("Failed to list of properties/namespace from global-zk", e);
+        }
+        return namespaces;
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/antiAffinity")
+    @ApiOperation(value = "Remove anti-affinity group of a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void removeNamespaceAntiAffinityGroup(@PathParam("property") String property,
+                                                 @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validatePoliciesReadOnlyAccess();
+
+        log.info("[{}] Deleting anti-affinity group for {}/{}/{}", clientAppId(), property, cluster, namespace);
+
+        try {
+            Stat nodeStat = new Stat();
+            final String path = path(POLICIES, property, cluster, namespace);
+            byte[] content = globalZk().getData(path, null, nodeStat);
+            Policies policies = jsonMapper().readValue(content, Policies.class);
+            policies.antiAffinityGroup = null;
+            globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
+            policiesCache().invalidate(path(POLICIES, property, cluster, namespace));
+            log.info("[{}] Successfully removed anti-affinity group for a namespace={}/{}/{}", clientAppId(), property,
+                    cluster, namespace);
+
+        } catch (KeeperException.NoNodeException e) {
+            log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: does not exist", clientAppId(),
+                    property, cluster, namespace);
+            throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
+        } catch (KeeperException.BadVersionException e) {
+            log.warn("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}: concurrent modification",
+                    clientAppId(), property, cluster, namespace);
+            throw new RestException(Status.CONFLICT, "Concurrent modification");
+        } catch (Exception e) {
+            log.error("[{}] Failed to remove anti-affinity group for namespace {}/{}/{}", clientAppId(), property,
+                    cluster, namespace, e);
+            throw new RestException(e);
+        }
+    }
+
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/deduplication")
+    @ApiOperation(hidden = true, value = "Enable or disable broker side deduplication for all topics in a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public void modifyDeduplication(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, boolean enableDeduplication) {
+        validateNamespaceName(property, cluster, namespace);
+        internalModifyDeduplication(enableDeduplication);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/bundles")
+    @ApiOperation(hidden = true, value = "Get the bundles split data.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") })
+    public BundlesData getBundlesData(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validatePoliciesReadOnlyAccess();
+        validateNamespaceName(property, cluster, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+
+        return policies.bundles;
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/unload")
+    @ApiOperation(hidden = true, value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
+            + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+            + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
+            + "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
+            + "result in non-persistent message loss and unexpected connection closure to the clients.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") })
+    public void unloadNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, cluster, namespace);
+        internalUnloadNamespace();
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/{bundle}/unload")
+    @ApiOperation(hidden = true, value = "Unload a namespace bundle")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalUnloadNamespaceBundle(bundleRange, authoritative);
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/{bundle}/split")
+    @ApiOperation(hidden = true, value = "Split a namespace bundle")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative,
+            @QueryParam("unload") @DefaultValue("false") boolean unload) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSplitNamespaceBundle(bundleRange, authoritative, unload);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/dispatchRate")
+    @ApiOperation(hidden = true, value = "Set dispatch-rate throttling for all topics of the namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void setDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, DispatchRate dispatchRate) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetDispatchRate(dispatchRate);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/dispatchRate")
+    @ApiOperation(hidden = true, value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public DispatchRate getDispatchRate(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, cluster, namespace);
+        return internalGetDispatchRate();
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/backlogQuotaMap")
+    @ApiOperation(hidden = true, value = "Get backlog quota map on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public Map<BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, cluster, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+        return policies.backlog_quota_map;
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/backlogQuota")
+    @ApiOperation(hidden = true, value = " Set a backlog quota for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") })
+    public void setBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType,
+            BacklogQuota backlogQuota) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetBacklogQuota(backlogQuotaType, backlogQuota);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/backlogQuota")
+    @ApiOperation(hidden = true, value = "Remove a backlog quota policy from a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void removeBacklogQuota(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace,
+            @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) {
+        validateNamespaceName(property, cluster, namespace);
+        internalRemoveBacklogQuota(backlogQuotaType);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/retention")
+    @ApiOperation(hidden = true, value = "Get retention config on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public RetentionPolicies getRetention(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, cluster, namespace);
+        return internalGetRetention();
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/retention")
+    @ApiOperation(hidden = true, value = " Set retention configuration on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") })
+    public void setRetention(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, RetentionPolicies retention) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetRetention(retention);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/persistence")
+    @ApiOperation(hidden = true, value = "Set the persistence configuration for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 400, message = "Invalid persistence policies") })
+    public void setPersistence(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, PersistencePolicies persistence) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetPersistence(persistence);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/persistence")
+    @ApiOperation(hidden = true, value = "Get the persistence configuration for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public PersistencePolicies getPersistence(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, cluster, namespace);
+        return internalGetPersistence();
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/clearBacklog")
+    @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalClearNamespaceBacklog(authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog")
+    @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBundleBacklog(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalClearNamespaceBundleBacklog(bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}")
+    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("subscription") String subscription,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalClearNamespaceBacklogForSubscription(subscription, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}")
+    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("subscription") String subscription, @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalClearNamespaceBundleBacklogForSubscription(subscription, bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}")
+    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalUnsubscribeNamespace(subscription, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}")
+    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, cluster, namespace);
+        internalUnsubscribeNamespaceBundle(subscription, bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode")
+    @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void setSubscriptionAuthMode(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetSubscriptionAuthMode(subscriptionAuthMode);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/encryptionRequired")
+    @ApiOperation(hidden = true, value = "Message encryption is required or not for all topics in a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"), })
+    public void modifyEncryptionRequired(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, boolean encryptionRequired) {
+        validateNamespaceName(property, cluster, namespace);
+        internalModifyEncryptionRequired(encryptionRequired);
+    }
+
+    private static final Logger log = LoggerFactory.getLogger(Namespaces.class);
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
similarity index 79%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
index 68b4e836d..c423f2184 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
@@ -16,15 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.admin;
+package org.apache.pulsar.broker.admin.v1;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
-import static org.apache.pulsar.common.util.Codec.decode;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.Encoded;
@@ -37,12 +31,12 @@
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 
+import com.google.common.collect.Lists;
 import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
 import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationDomain;
+import org.apache.pulsar.common.naming.Constants;
 import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceName;
@@ -55,13 +49,15 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
-
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
 /**
  */
 @Path("/non-persistent")
@@ -69,96 +65,94 @@
 @Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic")
 public class NonPersistentTopics extends PersistentTopics {
     private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
-    
+
     @GET
     @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Get partitioned topic metadata.")
+    @ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        return getPartitionedTopicMetadata(property, cluster, namespace, destination, authoritative);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return getPartitionedTopicMetadata(destinationName, authoritative);
     }
 
     @GET
     @Path("{property}/{cluster}/{namespace}/{destination}/stats")
-    @ApiOperation(value = "Get the stats for the topic.")
+    @ApiOperation(hidden = true, value = "Get the stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
-    public NonPersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
+    public NonPersistentTopicStats getStats(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminOperationOnDestination(dn, authoritative);
-        Topic topic = getTopicReference(dn);
-        return ((NonPersistentTopic)topic).getStats();
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnDestination(authoritative);
+        Topic topic = getTopicReference(destinationName);
+        return ((NonPersistentTopic) topic).getStats();
     }
 
     @GET
     @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
-    @ApiOperation(value = "Get the internal stats for the topic.")
+    @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String destination,
+            @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminOperationOnDestination(dn, authoritative);
-        Topic topic = getTopicReference(dn);
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnDestination(authoritative);
+        Topic topic = getTopicReference(destinationName);
         return topic.getInternalStats();
     }
 
     @PUT
     @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
-    @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
+    @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic already exist") })
     public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination, int numPartitions,
-            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        validateAdminAccessOnProperty(dn.getProperty());
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateAdminAccessOnProperty(destinationName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
-            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, property, cluster, namespace, domain(),
-                    dn.getEncodedLocalName());
+            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
+                    destinationName.getEncodedLocalName());
             byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
             zkCreateOptimistic(path, data);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), dn);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
         } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), dn);
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
             throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), dn, e);
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
             throw new RestException(e);
         }
     }
 
     @PUT
     @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
-    @ApiOperation(value = "Unload a topic")
+    @ApiOperation(hidden = true, value = "Unload a topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        log.info("[{}] Unloading topic {}/{}/{}/{}", clientAppId(), property, cluster, namespace, destination);
-        destination = decode(destination);
-        DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
         }
-        unloadTopic(dn, authoritative);
+        unloadTopic(destinationName, authoritative);
     }
 
     @GET
@@ -167,13 +161,13 @@ public void unloadTopic(@PathParam("property") String property, @PathParam("clus
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist") })
     public List<String> getList(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace) {
+                                @PathParam("namespace") String namespace) {
         log.info("[{}] list of topics on namespace {}/{}/{}/{}", clientAppId(), property, cluster, namespace);
         validateAdminAccessOnProperty(property);
         Policies policies = getNamespacePolicies(property, cluster, namespace);
         NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
 
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
+        if (!cluster.equals(Constants.GLOBAL_CLUSTER)) {
             validateClusterOwnership(cluster);
             validateClusterForProperty(property, cluster);
         } else {
@@ -213,19 +207,19 @@ public void unloadTopic(@PathParam("property") String property, @PathParam("clus
         }
         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<String> getListFromBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) {
+                                          @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) {
         log.info("[{}] list of topics on namespace bundle {}/{}/{}/{}", clientAppId(), property, cluster, namespace,
                 bundleRange);
         validateAdminAccessOnProperty(property);
         Policies policies = getNamespacePolicies(property, cluster, namespace);
-        if (!cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
+        if (!cluster.equals(Constants.GLOBAL_CLUSTER)) {
             validateClusterOwnership(cluster);
             validateClusterForProperty(property, cluster);
         } else {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
new file mode 100644
index 000000000..624a3f3da
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
@@ -0,0 +1,421 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v1;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.Encoded;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.container.AsyncResponse;
+import javax.ws.rs.container.Suspended;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.PartitionedTopicStats;
+import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
+import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
+import org.apache.pulsar.common.policies.data.PersistentTopicStats;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+/**
+ */
+@Path("/persistent")
+@Produces(MediaType.APPLICATION_JSON)
+@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
+public class PersistentTopics extends PersistentTopicsBase {
+    @GET
+    @Path("/{property}/{cluster}/{namespace}")
+    @ApiOperation(hidden = true, value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
+    public List<String> 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<String> 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<String, Set<AuthAction>> 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<AuthAction> actions) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalGrantPermissionsOnDestination(role, actions);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
+    @ApiOperation(hidden = true, value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+            + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
+    public void revokePermissionsOnDestination(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalRevokePermissionsOnDestination(role);
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Partitioned topic already exist") })
+    public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalCreatePartitionedTopic(numPartitions, authoritative);
+    }
+
+    /**
+     * It updates number of partitions of an existing non-global partitioned topic. It requires partitioned-topic to be
+     * already exist and number of new partitions must be greater than existing number of partitions. Decrementing
+     * number of partitions requires deletion of topic which is not supported.
+     *
+     * Already created partitioned producers and consumers can't see newly created partitions and it requires to
+     * recreate them at application so, newly created producers and consumers can connect to newly added partitions as
+     * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application.
+     *
+     * @param property
+     * @param cluster
+     * @param namespace
+     * @param numPartitions
+     */
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @ApiOperation(hidden = true, value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Partitioned topic does not exist") })
+    public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            int numPartitions) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalUpdatePartitionedTopic(numPartitions);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetPartitionedMetadata(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @ApiOperation(hidden = true, value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Partitioned topic does not exist") })
+    public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalDeletePartitionedTopic(authoritative);
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
+    @ApiOperation(hidden = true, value = "Unload a topic")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalUnloadTopic(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/{destination}")
+    @ApiOperation(hidden = true, value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist"),
+            @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
+    public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalDeleteTopic(authoritative);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions")
+    @ApiOperation(hidden = true, value = "Get the list of persistent subscriptions for a given topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public List<String> getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetSubscriptions(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{cluster}/{namespace}/{destination}/stats")
+    @ApiOperation(hidden = true, value = "Get the stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetStats(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
+    @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetInternalStats(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{cluster}/{namespace}/{destination}/internal-info")
+    @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @Suspended AsyncResponse asyncResponse) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalGetManagedLedgerInfo(asyncResponse);
+    }
+
+    @GET
+    @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats")
+    @ApiOperation(hidden = true, value = "Get the stats for the partitioned topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetPartitionedStats(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}")
+    @ApiOperation(hidden = true, value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist"),
+            @ApiResponse(code = 412, message = "Subscription has active consumers") })
+    public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalDeleteSubscription(subName, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all")
+    @ApiOperation(hidden = true, value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalSkipAllMessages(subName, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+    @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName, @PathParam("numMessages") int numMessages,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalSkipMessages(subName, numMessages, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+    @ApiOperation(hidden = true, value = "Expire messages on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalExpireMessages(subName, expireTimeInSeconds, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+    @ApiOperation(hidden = true, value = "Expire messages on all subscriptions of topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+    @ApiOperation(hidden = true, value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
+    public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName, @PathParam("timestamp") long timestamp,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalResetCursor(subName, timestamp, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+    @ApiOperation(hidden = true, value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
+            @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
+    public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        internalResetCursorOnPosition(subName, authoritative, messageId);
+    }
+
+    @PUT
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}")
+    @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
+            @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
+    public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
+           @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
+           @PathParam("subscriptionName") String subscriptionName,
+           @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
+        validateDestinationName(property, cluster, namespace, destination);
+        internalCreateSubscription(subscriptionName, messageId, authoritative);
+    }
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+    @ApiOperation(hidden = true, value = "Peek nth message on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
+    public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalPeekNthMessage(subName, messagePosition, authoritative);
+    }
+
+    @GET
+    @Path("{property}/{cluster}/{namespace}/{destination}/backlog")
+    @ApiOperation(hidden = true, value = "Get estimated backlog for offline topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalGetBacklog(authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{destination}/terminate")
+    @ApiOperation(hidden = true, value = "Terminate a topic. A topic that is terminated will not accept any more "
+            + "messages to be published and will let consumer to drain existing messages in backlog")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, cluster, namespace, encodedTopic);
+        return internalTerminate(authoritative);
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java
new file mode 100644
index 000000000..879a98d72
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Properties.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v1;
+
+import io.swagger.annotations.Api;
+import org.apache.pulsar.broker.admin.impl.PropertiesBase;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("/properties")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/properties", description = "PropertiesBase admin apis", tags = "properties")
+public class Properties extends PropertiesBase {
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java
new file mode 100644
index 000000000..fb77ee033
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/ResourceQuotas.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v1;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.pulsar.broker.admin.impl.ResourceQuotasBase;
+import org.apache.pulsar.common.policies.data.ResourceQuota;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+@Path("/resource-quotas")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas")
+public class ResourceQuotas extends ResourceQuotasBase {
+
+    @GET
+    @Path("/{property}/{cluster}/{namespace}/{bundle}")
+    @ApiOperation(hidden = true, value = "Get resource quota of a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange) {
+        validateNamespaceName(property, cluster, namespace);
+        return internalGetNamespaceBundleResourceQuota(bundleRange);
+    }
+
+    @POST
+    @Path("/{property}/{cluster}/{namespace}/{bundle}")
+    @ApiOperation(hidden = true, value = "Set resource quota on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void setNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange, ResourceQuota quota) {
+        validateNamespaceName(property, cluster, namespace);
+        internalSetNamespaceBundleResourceQuota(bundleRange, quota);
+    }
+
+    @DELETE
+    @Path("/{property}/{cluster}/{namespace}/{bundle}")
+    @ApiOperation(hidden = true, value = "Remove resource quota for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void removeNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange) {
+        validateNamespaceName(property, cluster, namespace);
+        internalRemoveNamespaceBundleResourceQuota(bundleRange);
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java
new file mode 100644
index 000000000..f0318d7e9
--- /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 000000000..7a69bb80e
--- /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 000000000..f51dad865
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Clusters.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import io.swagger.annotations.Api;
+import org.apache.pulsar.broker.admin.impl.ClustersBase;
+
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("/clusters")
+@Api(value = "/clusters", description = "Cluster admin apis", tags = "clusters")
+@Produces(MediaType.APPLICATION_JSON)
+public class Clusters extends ClustersBase {
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
new file mode 100644
index 000000000..d00499a4b
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -0,0 +1,519 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.pulsar.broker.admin.impl.NamespacesBase;
+import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.BacklogQuota;
+import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
+import org.apache.pulsar.common.policies.data.BundlesData;
+import org.apache.pulsar.common.policies.data.DispatchRate;
+import org.apache.pulsar.common.policies.data.PersistencePolicies;
+import org.apache.pulsar.common.policies.data.Policies;
+import org.apache.pulsar.common.policies.data.RetentionPolicies;
+import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+@Path("/namespaces")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
+public class Namespaces extends NamespacesBase {
+
+    @GET
+    @Path("/{property}")
+    @ApiOperation(value = "Get the list of all the namespaces for a certain property.", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property doesn't exist") })
+    public List<String> getPropertyNamespaces(@PathParam("property") String property) {
+        return internalGetPropertyNamespaces(property);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/topics")
+    @ApiOperation(value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public List<String> getDestinations(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+
+        // Validate that namespace exists, throws 404 if it doesn't exist
+        getNamespacePolicies(namespaceName);
+
+        try {
+            return pulsar().getNamespaceService().getListOfDestinations(namespaceName);
+        } catch (Exception e) {
+            log.error("Failed to get topics list for namespace {}", namespaceName, e);
+            throw new RestException(e);
+        }
+    }
+
+    @GET
+    @Path("/{property}/{namespace}")
+    @ApiOperation(value = "Get the dump all the policies specified for a namespace.", response = Policies.class)
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public Policies getPolicies(@PathParam("property") String property, @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+        return getNamespacePolicies(namespaceName);
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}")
+    @ApiOperation(value = "Creates a new namespace with the specified policies")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster doesn't exist"),
+            @ApiResponse(code = 409, message = "Namespace already exists"),
+            @ApiResponse(code = 412, message = "Namespace name is not valid") })
+    public void createNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            Policies policies) {
+        validateNamespaceName(property, namespace);
+
+        policies = getDefaultPolicesIfNull(policies);
+        internalCreateNamespace(policies);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}")
+    @ApiOperation(value = "Delete a namespace and all the topics under it.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Namespace is not empty") })
+    public void deleteNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalDeleteNamespace(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/bundle/{bundle}")
+    @ApiOperation(value = "Delete a namespace bundle and all the topics under it.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Namespace bundle is not empty") })
+    public void deleteNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalDeleteNamespaceBundle(bundleRange, authoritative);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/permissions")
+    @ApiOperation(value = "Retrieve the permissions for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Namespace is not empty") })
+    public Map<String, Set<AuthAction>> getPermissions(@PathParam("property") String property,
+            @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+        return policies.auth_policies.namespace_auth;
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/permissions/{role}")
+    @ApiOperation(value = "Grant a new permission to a role on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void grantPermissionOnNamespace(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("role") String role, Set<AuthAction> actions) {
+        validateNamespaceName(property, namespace);
+        internalGrantPermissionOnNamespace(role, actions);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/permissions/{role}")
+    @ApiOperation(value = "Revoke all permissions to a role on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public void revokePermissionsOnNamespace(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("role") String role) {
+        validateNamespaceName(property, namespace);
+        internalRevokePermissionsOnNamespace(role);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/replication")
+    @ApiOperation(value = "Get the replication clusters for a namespace.", response = String.class, responseContainer = "List")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Namespace is not global") })
+    public List<String> getNamespaceReplicationClusters(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+
+        return internalGetNamespaceReplicationClusters();
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/replication")
+    @ApiOperation(value = "Set the replication clusters for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"),
+            @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") })
+    public void setNamespaceReplicationClusters(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, List<String> clusterIds) {
+        validateNamespaceName(property, namespace);
+        internalSetNamespaceReplicationClusters(clusterIds);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/messageTTL")
+    @ApiOperation(value = "Get the message TTL for the namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public int getNamespaceMessageTTL(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+        return policies.message_ttl_in_seconds;
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/messageTTL")
+    @ApiOperation(value = "Set message TTL in seconds for namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Invalid TTL") })
+    public void setNamespaceMessageTTL(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            int messageTTL) {
+        validateNamespaceName(property, namespace);
+        internalSetNamespaceMessageTTL(messageTTL);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/deduplication")
+    @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
+    public void modifyDeduplication(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            boolean enableDeduplication) {
+        validateNamespaceName(property, namespace);
+        internalModifyDeduplication(enableDeduplication);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/bundles")
+    @ApiOperation(value = "Get the bundles split data.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") })
+    public BundlesData getBundlesData(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validatePoliciesReadOnlyAccess();
+        validateNamespaceName(property, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+
+        return policies.bundles;
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/unload")
+    @ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
+            + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+            + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
+            + "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
+            + "result in non-persistent message loss and unexpected connection closure to the clients.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Namespace is already unloaded or Namespace has bundles activated") })
+    public void unloadNamespace(@PathParam("property") String property, @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        internalUnloadNamespace();
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{bundle}/unload")
+    @ApiOperation(value = "Unload a namespace bundle")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void unloadNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalUnloadNamespaceBundle(bundleRange, authoritative);
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{bundle}/split")
+    @ApiOperation(value = "Split a namespace bundle")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void splitNamespaceBundle(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative,
+            @QueryParam("unload") @DefaultValue("false") boolean unload) {
+        validateNamespaceName(property, namespace);
+        internalSplitNamespaceBundle(bundleRange, authoritative, unload);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/dispatchRate")
+    @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void setDispatchRate(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            DispatchRate dispatchRate) {
+        validateNamespaceName(property, namespace);
+        internalSetDispatchRate(dispatchRate);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/dispatchRate")
+    @ApiOperation(value = "Get dispatch-rate configured for the namespace, -1 represents not configured yet")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public DispatchRate getDispatchRate(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        return internalGetDispatchRate();
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/backlogQuotaMap")
+    @ApiOperation(value = "Get backlog quota map on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public Map<BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateAdminAccessOnProperty(property);
+        validateNamespaceName(property, namespace);
+
+        Policies policies = getNamespacePolicies(namespaceName);
+        return policies.backlog_quota_map;
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/backlogQuota")
+    @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 412, message = "Specified backlog quota exceeds retention quota. Increase retention quota and retry request") })
+    public void setBacklogQuota(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, BacklogQuota backlogQuota) {
+        validateNamespaceName(property, namespace);
+        internalSetBacklogQuota(backlogQuotaType, backlogQuota);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/backlogQuota")
+    @ApiOperation(value = "Remove a backlog quota policy from a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void removeBacklogQuota(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType) {
+        validateNamespaceName(property, namespace);
+        internalRemoveBacklogQuota(backlogQuotaType);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/retention")
+    @ApiOperation(value = "Get retention config on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public RetentionPolicies getRetention(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        return internalGetRetention();
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/retention")
+    @ApiOperation(value = " Set retention configuration on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") })
+    public void setRetention(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            RetentionPolicies retention) {
+        validateNamespaceName(property, namespace);
+        internalSetRetention(retention);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/persistence")
+    @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"),
+            @ApiResponse(code = 400, message = "Invalid persistence policies") })
+    public void setPersistence(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            PersistencePolicies persistence) {
+        validateNamespaceName(property, namespace);
+        internalSetPersistence(persistence);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/persistence")
+    @ApiOperation(value = "Get the persistence configuration for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public PersistencePolicies getPersistence(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        return internalGetPersistence();
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/clearBacklog")
+    @ApiOperation(value = "Clear backlog for all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalClearNamespaceBacklog(authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{bundle}/clearBacklog")
+    @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBundleBacklog(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalClearNamespaceBundleBacklog(bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/clearBacklog/{subscription}")
+    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalClearNamespaceBacklogForSubscription(subscription, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{bundle}/clearBacklog/{subscription}")
+    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalClearNamespaceBundleBacklogForSubscription(subscription, bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/unsubscribe/{subscription}")
+    @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalUnsubscribeNamespace(subscription, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{bundle}/unsubscribe/{subscription}")
+    @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public void unsubscribeNamespaceBundle(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("subscription") String subscription,
+            @PathParam("bundle") String bundleRange,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateNamespaceName(property, namespace);
+        internalUnsubscribeNamespaceBundle(subscription, bundleRange, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/subscriptionAuthMode")
+    @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void setSubscriptionAuthMode(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, SubscriptionAuthMode subscriptionAuthMode) {
+        validateNamespaceName(property, namespace);
+        internalSetSubscriptionAuthMode(subscriptionAuthMode);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/encryptionRequired")
+    @ApiOperation(value = "Message encryption is required or not for all topics in a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification"), })
+    public void modifyEncryptionRequired(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, boolean encryptionRequired) {
+        validateNamespaceName(property, namespace);
+        internalModifyEncryptionRequired(encryptionRequired);
+    }
+
+    private Policies getDefaultPolicesIfNull(Policies policies) {
+        if (policies != null) {
+            return policies;
+        }
+
+        Policies defaultPolicies = new Policies();
+        int defaultNumberOfBundles = config().getDefaultNumberOfNamespaceBundles();
+        defaultPolicies.bundles = getBundles(defaultNumberOfBundles);
+        return defaultPolicies;
+    }
+
+    private static final Logger log = LoggerFactory.getLogger(Namespaces.class);
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
new file mode 100644
index 000000000..55bd9be9b
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.Encoded;
+import javax.ws.rs.GET;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.pulsar.broker.service.Topic;
+import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
+import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
+import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+/**
+ */
+@Path("/non-persistent")
+@Produces(MediaType.APPLICATION_JSON)
+@Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic")
+public class NonPersistentTopics extends PersistentTopics {
+    private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
+
+    @GET
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Get partitioned topic metadata.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return getPartitionedTopicMetadata(destinationName, authoritative);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/stats")
+    @ApiOperation(value = "Get the stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public NonPersistentTopicStats getStats(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        validateAdminOperationOnDestination(destinationName, authoritative);
+        Topic topic = getTopicReference(destinationName);
+        return ((NonPersistentTopic) topic).getStats();
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/internalStats")
+    @ApiOperation(value = "Get the internal stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        validateAdminOperationOnDestination(destinationName, authoritative);
+        Topic topic = getTopicReference(destinationName);
+        return topic.getInternalStats();
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Partitioned topic already exist") })
+    public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        validateAdminAccessOnProperty(destinationName.getProperty());
+        if (numPartitions <= 1) {
+            throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
+        }
+        try {
+            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
+                    destinationName.getEncodedLocalName());
+            byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
+            zkCreateOptimistic(path, data);
+            // we wait for the data to be synced in all quorums and the observers
+            Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+        } catch (KeeperException.NodeExistsException e) {
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+            throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
+        } catch (Exception e) {
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+            throw new RestException(e);
+        }
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{destination}/unload")
+    @ApiOperation(value = "Unload a topic")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+        if (destinationName.isGlobal()) {
+            validateGlobalNamespaceOwnership(namespaceName);
+        }
+        unloadTopic(destinationName, authoritative);
+    }
+
+    protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
+        validateAdminAccessOnProperty(fqdn.getProperty());
+        validateDestinationOwnership(fqdn, authoritative);
+    }
+
+    private Topic getTopicReference(DestinationName dn) {
+        try {
+            Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+            checkNotNull(topic);
+            return topic;
+        } catch (Exception e) {
+            throw new RestException(Status.NOT_FOUND, "Topic not found");
+        }
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java
new file mode 100644
index 000000000..b52a5add9
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java
@@ -0,0 +1,396 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.Encoded;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.container.AsyncResponse;
+import javax.ws.rs.container.Suspended;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.PartitionedTopicStats;
+import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
+import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
+import org.apache.pulsar.common.policies.data.PersistentTopicStats;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+/**
+ */
+@Path("/persistent")
+@Produces(MediaType.APPLICATION_JSON)
+@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
+public class PersistentTopics extends PersistentTopicsBase {
+
+    @GET
+    @Path("/{property}/{namespace}")
+    @ApiOperation(value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
+    public List<String> getList(@PathParam("property") String property, @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        return internalGetList();
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/partitioned")
+    @ApiOperation(value = "Get the list of partitioned topics under a namespace.", response = String.class, responseContainer = "List")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
+    public List<String> getPartitionedTopicList(@PathParam("property") String property,
+            @PathParam("namespace") String namespace) {
+        validateNamespaceName(property, namespace);
+        return internalGetPartitionedTopicList();
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/{destination}/permissions")
+    @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
+            + "namespace level combined (union) with any eventual specific permission set on the destination.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist") })
+    public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetPermissionsOnDestination();
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/permissions/{role}")
+    @ApiOperation(value = "Grant a new permission to a role on a single topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void grantPermissionsOnDestination(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("role") String role, Set<AuthAction> actions) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalGrantPermissionsOnDestination(role, actions);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/{destination}/permissions/{role}")
+    @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+            + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace doesn't exist"),
+            @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
+    public void revokePermissionsOnDestination(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("role") String role) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalRevokePermissionsOnDestination(role);
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Partitioned topic already exist") })
+    public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalCreatePartitionedTopic(numPartitions, authoritative);
+    }
+
+    /**
+     * It updates number of partitions of an existing non-global partitioned topic. It requires partitioned-topic to be
+     * already exist and number of new partitions must be greater than existing number of partitions. Decrementing
+     * number of partitions requires deletion of topic which is not supported.
+     *
+     * Already created partitioned producers and consumers can't see newly created partitions and it requires to
+     * recreate them at application so, newly created producers and consumers can connect to newly added partitions as
+     * well. Therefore, it can violate partition ordering at producers until all producers are restarted at application.
+     *
+     * @param property
+     * @param cluster
+     * @param namespace
+     * @param destination
+     * @param numPartitions
+     */
+    @POST
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Partitioned topic does not exist") })
+    public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, int numPartitions) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalUpdatePartitionedTopic(numPartitions);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Get partitioned topic metadata.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetPartitionedMetadata(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/{destination}/partitions")
+    @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Partitioned topic does not exist") })
+    public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalDeletePartitionedTopic(authoritative);
+    }
+
+    @PUT
+    @Path("/{property}/{namespace}/{destination}/unload")
+    @ApiOperation(value = "Unload a topic")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalUnloadTopic(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/{destination}")
+    @ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist"),
+            @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
+    public void deleteTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalDeleteTopic(authoritative);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/{destination}/subscriptions")
+    @ApiOperation(value = "Get the list of persistent subscriptions for a given topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public List<String> getSubscriptions(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetSubscriptions(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/stats")
+    @ApiOperation(value = "Get the stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PersistentTopicStats getStats(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetStats(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/internalStats")
+    @ApiOperation(value = "Get the internal stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetInternalStats(authoritative);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/internal-info")
+    @ApiOperation(value = "Get the internal stats for the topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalGetManagedLedgerInfo(asyncResponse);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/partitioned-stats")
+    @ApiOperation(value = "Get the stats for the partitioned topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetPartitionedStats(authoritative);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}")
+    @ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic does not exist"),
+            @ApiResponse(code = 412, message = "Subscription has active consumers") })
+    public void deleteSubscription(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalDeleteSubscription(subName, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip_all")
+    @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void skipAllMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalSkipAllMessages(subName, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+    @ApiOperation(value = "Skip messages on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void skipMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("numMessages") int numMessages,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalSkipMessages(subName, numMessages, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+    @ApiOperation(value = "Expire messages on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void expireTopicMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalExpireMessages(subName, expireTimeInSeconds, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+    @ApiOperation(value = "Expire messages on all subscriptions of topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
+    public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+    @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
+    public void resetCursor(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("timestamp") long timestamp,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalResetCursor(subName, timestamp, authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+    @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
+            @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
+    public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
+        validateDestinationName(property, namespace, encodedTopic);
+        internalResetCursorOnPosition(subName, authoritative, messageId);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+    @ApiOperation(value = "Peek nth message on a topic subscription.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
+    public Response peekNthMessage(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("messagePosition") int messagePosition,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalPeekNthMessage(subName, messagePosition, authoritative);
+    }
+
+    @GET
+    @Path("{property}/{namespace}/{destination}/backlog")
+    @ApiOperation(value = "Get estimated backlog for offline topic.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalGetBacklog(authoritative);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{destination}/terminate")
+    @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more "
+            + "messages to be published and will let consumer to drain existing messages in backlog")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
+            @ApiResponse(code = 404, message = "Topic does not exist") })
+    public MessageId terminate(@PathParam("property") String property, @PathParam("namespace") String namespace,
+            @PathParam("destination") @Encoded String encodedTopic,
+            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
+        validateDestinationName(property, namespace, encodedTopic);
+        return internalTerminate(authoritative);
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java
new file mode 100644
index 000000000..19d36520f
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Properties.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import io.swagger.annotations.Api;
+import org.apache.pulsar.broker.admin.impl.PropertiesBase;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("/properties")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/properties", description = "PropertiesBase admin apis", tags = "properties")
+public class Properties extends PropertiesBase {
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java
new file mode 100644
index 000000000..b84a14ff0
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.admin.v2;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.pulsar.broker.admin.impl.ResourceQuotasBase;
+import org.apache.pulsar.common.policies.data.ResourceQuota;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiResponse;
+import io.swagger.annotations.ApiResponses;
+
+@Path("/resource-quotas")
+@Produces(MediaType.APPLICATION_JSON)
+@Consumes(MediaType.APPLICATION_JSON)
+@Api(value = "/resource-quotas", description = "Quota admin APIs", tags = "resource-quotas")
+public class ResourceQuotas extends ResourceQuotasBase {
+
+    @GET
+    @ApiOperation(value = "Get the default quota", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public ResourceQuota getDefaultResourceQuota() throws Exception {
+        return super.getDefaultResourceQuota();
+    }
+
+    @POST
+    @ApiOperation(value = "Set the default quota", response = String.class, responseContainer = "Set")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
+    public void setDefaultResourceQuota(ResourceQuota quota) throws Exception {
+        super.setDefaultResourceQuota(quota);
+    }
+
+    @GET
+    @Path("/{property}/{namespace}/{bundle}")
+    @ApiOperation(value = "Get resource quota of a namespace bundle.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Namespace does not exist") })
+    public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) {
+        validateNamespaceName(property, namespace);
+        return internalGetNamespaceBundleResourceQuota(bundleRange);
+    }
+
+    @POST
+    @Path("/{property}/{namespace}/{bundle}")
+    @ApiOperation(value = "Set resource quota on a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void setNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange, ResourceQuota quota) {
+        validateNamespaceName(property, namespace);
+        internalSetNamespaceBundleResourceQuota(bundleRange, quota);
+    }
+
+    @DELETE
+    @Path("/{property}/{namespace}/{bundle}")
+    @ApiOperation(value = "Remove resource quota for a namespace.")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 409, message = "Concurrent modification") })
+    public void removeNamespaceBundleResourceQuota(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("bundle") String bundleRange) {
+        validateNamespaceName(property, namespace);
+        internalRemoveNamespaceBundleResourceQuota(bundleRange);
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
index 19d3f348e..80cb31cb1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java
@@ -253,7 +253,7 @@ public static boolean isLoadSheddingEnabled(final PulsarService pulsar) {
      * @param assignedBundleName
      *            Name of bundle to be assigned.
      * @param candidates
-     *            Brokers available for placement.
+     *            BrokersBase available for placement.
      * @param brokerToNamespaceToBundleRange
      *            Map from brokers to namespaces to bundle ranges.
      */
@@ -302,14 +302,14 @@ public static void removeMostServicingBrokersForNamespace(final String assignedB
      * eg.
      * <pre>
      * Before:
-     * Domain-count  Brokers-count
+     * Domain-count  BrokersBase-count
      * ____________  ____________
      * d1-3          b1-2,b2-1
      * d2-3          b3-2,b4-1
      * d3-4          b5-2,b6-2
      *
      * After filtering: "candidates" brokers
-     * Domain-count  Brokers-count
+     * Domain-count  BrokersBase-count
      * ____________  ____________
      * d1-3          b2-1
      * d2-3          b4-1
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
index 0393d6a36..300bd84e9 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
@@ -52,6 +52,7 @@
 import org.apache.pulsar.common.naming.DestinationDomain;
 import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.util.Codec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -211,7 +212,8 @@ public String getNamespaceBundle(@PathParam("destination-domain") String destina
             if (differentClusterData != null) {
                 if (log.isDebugEnabled()) {
                     log.debug("[{}] Redirecting the lookup call to {}/{} cluster={}", clientAppId,
-                            differentClusterData.getBrokerServiceUrl(), differentClusterData.getBrokerServiceUrlTls(), cluster);
+                            differentClusterData.getBrokerServiceUrl(), differentClusterData.getBrokerServiceUrlTls(),
+                            cluster);
                 }
                 validationFuture.complete(newLookupResponse(differentClusterData.getBrokerServiceUrl(),
                         differentClusterData.getBrokerServiceUrlTls(), true, LookupType.Redirect, requestId, false));
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
index e7b885a42..5319344b4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
@@ -726,16 +726,18 @@ public ServiceUnitId getServiceUnitId(DestinationName destinationName) throws Ex
         return getBundle(destinationName);
     }
 
-    public List<String> getListOfDestinations(String property, String cluster, String namespace) throws Exception {
+    public List<String> getListOfDestinations(NamespaceName namespaceName) throws Exception {
         List<String> destinations = Lists.newArrayList();
 
         // For every topic there will be a managed ledger created.
         try {
-            String path = String.format("/managed-ledgers/%s/%s/%s/persistent", property, cluster, namespace);
-            LOG.debug("Getting children from managed-ledgers now: {}", path);
+            String path = String.format("/managed-ledgers/%s/persistent", namespaceName);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Getting children from managed-ledgers now: {}", path);
+            }
+
             for (String destination : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
-                destinations.add(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace,
-                        Codec.decode(destination)));
+                destinations.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(destination)));
             }
         } catch (KeeperException.NoNodeException e) {
             // NoNode means there are no persistent topics for this namespace
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitZkUtils.java
index 210b33507..a92585789 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<Long> 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<Long> 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<Long> range = getHashRange(parts[4]);
+            return factory.getBundle(NamespaceName.get(parts[2], parts[3]), range);
+        }
     }
 
     private static Range<Long> 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 2f3b27a75..1eff83b0e 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 void openLedgerFailed(ManagedLedgerException exception, Object ctx) {
             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 b50eee6ec..0b8d51280 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -20,7 +20,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
-import static org.apache.pulsar.broker.admin.PersistentTopics.getPartitionedTopicMetadata;
+import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.getPartitionedTopicMetadata;
 import static org.apache.pulsar.broker.lookup.DestinationLookup.lookupDestinationAsync;
 import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
 import static org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion.v5;
@@ -278,7 +278,7 @@ protected void handleLookup(CommandLookupTopic lookup) {
                 return null;
             }).exceptionally(ex -> {
                 final String msg = "Exception occured while trying to authorize lookup";
-                log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, authRole, topicName);
+                log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, authRole, topicName, ex);
                 ctx.writeAndFlush(newLookupErrorResponse(ServerError.AuthorizationError, msg, requestId));
                 lookupSemaphore.release();
                 return null;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
index c17182c54..60031fcbd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
@@ -45,14 +45,10 @@
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.admin.AdminResource;
-import org.apache.pulsar.broker.admin.Namespaces;
 import org.apache.pulsar.broker.authentication.AuthenticationDataHttps;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.namespace.NamespaceService;
-import org.apache.pulsar.common.naming.DestinationName;
-import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceBundles;
-import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.*;
 import org.apache.pulsar.common.policies.data.BundlesData;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.Policies;
@@ -277,12 +273,13 @@ private URI getRedirectionUrl(ClusterData differentClusterData) throws Malformed
     }
 
     protected static CompletableFuture<ClusterData> getClusterDataIfDifferentCluster(PulsarService pulsar,
-            String cluster, String clientAppId) {
+         String cluster, String clientAppId) {
 
         CompletableFuture<ClusterData> clusterDataFuture = new CompletableFuture<>();
 
         if (!isValidCluster(pulsar, cluster)) {
             try {
+                // this code should only happen with a v1 namespace format prop/cluster/namespaces
                 if (!pulsar.getConfiguration().getClusterName().equals(cluster)) {
                     // redirect to the cluster requested
                     pulsar.getConfigurationCache().clustersCache().getAsync(path("clusters", cluster))
@@ -310,15 +307,15 @@ private URI getRedirectionUrl(ClusterData differentClusterData) throws Malformed
         return clusterDataFuture;
     }
 
-    protected static boolean isValidCluster(PulsarService pulsarSevice, String cluster) {// If the cluster name is
-        // "global", don't validate the
-        // cluster ownership.
+    static boolean isValidCluster(PulsarService pulsarService, String cluster) {// If the cluster name is
+        // cluster == null or "global", don't validate the
+        // cluster ownership. Cluster will be null in v2 naming.
         // The validation will be done by checking the namespace configuration
-        if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
+        if (cluster == null || Constants.GLOBAL_CLUSTER.equals(cluster)) {
             return true;
         }
 
-        if (!pulsarSevice.getConfiguration().isAuthorizationEnabled()) {
+        if (!pulsarService.getConfiguration().isAuthorizationEnabled()) {
             // Without authorization, any cluster name should be valid and accepted by the broker
             return true;
         }
@@ -565,8 +562,7 @@ protected void validateGlobalNamespaceOwnership(NamespaceName namespace) {
         }
         final CompletableFuture<ClusterData> validationFuture = new CompletableFuture<>();
         final String localCluster = pulsarService.getConfiguration().getClusterName();
-        final String path = AdminResource.path(POLICIES, namespace.getProperty(), namespace.getCluster(),
-                namespace.getLocalName());
+        final String path = AdminResource.path(POLICIES, namespace.toString());
 
         pulsarService.getConfigurationCache().policiesCache().getAsync(path).thenAccept(policiesResult -> {
             if (policiesResult.isPresent()) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
index 27e3992b0..fd4d7ed9c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/PulsarBrokerVersionStringUtils.java
@@ -70,11 +70,11 @@ public static String fixVersionString(String version) {
     }
 
     /**
-     * Looks for a resource in the jar which is expected to be a java.util.Properties, then
+     * Looks for a resource in the jar which is expected to be a java.util.PropertiesBase, then
      * extract a specific property value.
      *
      * @return the property value, or null if the resource does not exist or the resource
-     *         is not a valid java.util.Properties or the resource does not contain the
+     *         is not a valid java.util.PropertiesBase or the resource does not contain the
      *         named property
      */
     private static String getPropertyFromResource(String resource, String propertyName) {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index 444d224a3..503811633 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -1493,7 +1493,7 @@ public void testObjectWithUnknowProperties() {
 
     /**
      * <pre>
-     * Verify: PersistentTopics.expireMessages()/expireMessagesForAllSubscriptions()
+     * Verify: PersistentTopicsBase.expireMessages()/expireMessagesForAllSubscriptions()
      * 1. Created multiple shared subscriptions and publisher on topic
      * 2. Publish messages on the topic
      * 3. expire message on sub-1 : backlog for sub-1 must be 0
@@ -1553,7 +1553,7 @@ public void testPersistentTopicsExpireMessages() throws Exception {
     }
 
     /**
-     * Verify: PersistentTopics.expireMessages()/expireMessagesForAllSubscriptions() for PartitionTopic
+     * Verify: PersistentTopicsBase.expireMessages()/expireMessagesForAllSubscriptions() for PartitionTopic
      *
      * @throws Exception
      */
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
index c587033af..52d118647 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
@@ -48,11 +48,18 @@
 
 import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.pulsar.broker.admin.v1.BrokerStats;
+import org.apache.pulsar.broker.admin.v1.Brokers;
+import org.apache.pulsar.broker.admin.v1.Clusters;
+import org.apache.pulsar.broker.admin.v1.Properties;
+import org.apache.pulsar.broker.admin.v1.Namespaces;
+import org.apache.pulsar.broker.admin.v1.PersistentTopics;
+import org.apache.pulsar.broker.admin.v1.ResourceQuotas;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.broker.loadbalance.ResourceUnit;
 import org.apache.pulsar.broker.web.PulsarWebResource;
 import org.apache.pulsar.broker.web.RestException;
+import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType;
@@ -90,7 +97,6 @@
     private BrokerStats brokerStats;
 
     private Field uriField;
-    private UriInfo uriInfo;
     private final String configClusterName = "use";
 
     public AdminTest() {
@@ -146,7 +152,6 @@ public void setup() throws Exception {
 
         uriField = PulsarWebResource.class.getDeclaredField("uri");
         uriField.setAccessible(true);
-        uriInfo = mock(UriInfo.class);
 
         persistentTopics = spy(new PersistentTopics());
         persistentTopics.setServletContext(new MockServletContext());
@@ -511,7 +516,7 @@ void resourceQuotas() throws Exception {
         String namespace = "ns";
         String bundleRange = "0x00000000_0xffffffff";
         Policies policies = new Policies();
-        doReturn(policies).when(resourceQuotas).getNamespacePolicies(property, cluster, namespace);
+        doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
         doReturn("client-id").when(resourceQuotas).clientAppId();
 
         try {
@@ -570,8 +575,7 @@ void brokerStats() throws Exception {
         StreamingOutput destination = brokerStats.getDestinations2();
         assertNotNull(destination);
         try {
-            Map<Long, Collection<ResourceUnit>> resource = brokerStats.getBrokerResourceAvailability("prop", "use",
-                    "ns2");
+            brokerStats.getBrokerResourceAvailability("prop", "use", "ns2");
             fail("should have failed as ModularLoadManager doesn't support it");
         } catch (RestException re) {
             // Ok
@@ -586,13 +590,12 @@ void persistentTopics() throws Exception {
         final String namespace = "ns";
         final String destination = "ds1";
         Policies policies = new Policies();
-        doReturn(policies).when(resourceQuotas).getNamespacePolicies(property, cluster, namespace);
+        doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
         doReturn("client-id").when(resourceQuotas).clientAppId();
         // create policies
         PropertyAdmin admin = new PropertyAdmin();
         admin.getAllowedClusters().add(cluster);
-        ZkUtils.createFullPathOptimistic(mockZookKeeper,
-                PulsarWebResource.path(POLICIES, property, cluster, namespace),
+        ZkUtils.createFullPathOptimistic(mockZookKeeper, PulsarWebResource.path(POLICIES, property, cluster, namespace),
                 ObjectMapperFactory.getThreadLocal().writeValueAsBytes(new Policies()), ZooDefs.Ids.OPEN_ACL_UNSAFE,
                 CreateMode.PERSISTENT);
 
@@ -601,8 +604,8 @@ void persistentTopics() throws Exception {
         // create destination
         assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList());
         persistentTopics.createPartitionedTopic(property, cluster, namespace, destination, 5, false);
-        assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList(
-                String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
+        assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists
+                .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
 
         CountDownLatch notificationLatch = new CountDownLatch(2);
         configurationCache.policiesCache().registerListener((path, data, stat) -> {
@@ -635,5 +638,5 @@ public void testRestExceptionMessage() {
         assertEquals(exception.getMessage(), message);
 
     }
-    
+
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
index 36c9ec2b7..f9ff65237 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
@@ -47,6 +47,8 @@
 import javax.ws.rs.core.UriInfo;
 
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.pulsar.broker.admin.v1.Namespaces;
+import org.apache.pulsar.broker.admin.v1.PersistentTopics;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.namespace.NamespaceEphemeralData;
 import org.apache.pulsar.broker.namespace.NamespaceService;
@@ -1064,20 +1066,16 @@ public void testValidateDestinationOwnership() throws Exception {
             doReturn(false).when(topics).isRequestHttps();
             doReturn("test").when(topics).clientAppId();
             mockWebUrl(localWebServiceUrl, testNs);
+            doReturn("persistent").when(topics).domain();
 
             try {
-                topics.validateAdminOperationOnDestination(topicName, false);
+                topics.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
+                        topicName.getNamespacePortion(), topicName.getEncodedLocalName());
+                topics.validateAdminOperationOnDestination(false);
             } catch (RestException e) {
                 fail("validateAdminAccessOnProperty failed");
             }
 
-            try {
-                topics.validateAdminOperationOnDestination(DestinationName.get(""), false);
-                fail("validateAdminAccessOnProperty failed");
-            } catch (Exception e) {
-                // OK
-            }
-
         } catch (RestException e) {
             fail("validateAdminAccessOnProperty failed");
         }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java
new file mode 100644
index 000000000..aed20fa55
--- /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 350501071..ef40ced32 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/DestinationName.java
@@ -18,8 +18,6 @@
  */
 package org.apache.pulsar.common.naming;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -64,6 +62,16 @@ public DestinationName load(String name) throws Exception {
                 }
             });
 
+    public static DestinationName get(String domain, NamespaceName namespaceName, String destination) {
+        String name = domain + "://" + namespaceName.toString() + '/' + destination;
+        return DestinationName.get(name);
+    }
+
+    public static DestinationName get(String domain, String property, String namespace, String destination) {
+        String name = domain + "://" + property + '/' + namespace + '/' + destination;
+        return DestinationName.get(name);
+    }
+
     public static DestinationName get(String domain, String property, String cluster, String namespace,
             String destination) {
         String name = domain + "://" + property + '/' + cluster + '/' + namespace + '/' + destination;
@@ -92,7 +100,9 @@ public static boolean isValid(String destination) {
     private DestinationName(String destination) {
         this.destination = destination;
         try {
-            // persistent://property/cluster/namespace/topic
+            // The topic name can be in two different forms:
+            // new:    persistent://property/namespace/topic
+            // legacy: persistent://property/cluster/namespace/topic
             if (!destination.contains("://")) {
                 throw new IllegalArgumentException(
                         "Invalid destination name: " + destination + " -- Domain is missing");
@@ -102,29 +112,44 @@ private DestinationName(String destination) {
             this.domain = DestinationDomain.getEnum(parts.get(0));
 
             String rest = parts.get(1);
-            // property/cluster/namespace/<localName>
+
+            // The rest of the name can be in different forms:
+            // new:    property/namespace/<localName>
+            // legacy: property/cluster/namespace/<localName>
             // Examples of localName:
             // 1. some/name/xyz//
             // 2. /xyz-123/feeder-2
+
+
             parts = Splitter.on("/").limit(4).splitToList(rest);
-            if (parts.size() != 4) {
+            if (parts.size() == 3) {
+                // New topic name without cluster name
+                this.property = parts.get(0);
+                this.cluster = null;
+                this.namespacePortion = parts.get(1);
+                this.localName = parts.get(2);
+                this.partitionIndex = getPartitionIndex(destination);
+                this.namespaceName = NamespaceName.get(property, namespacePortion);
+            } else if (parts.size() == 4) {
+                // Legacy topic name that includes cluster name
+                this.property = parts.get(0);
+                this.cluster = parts.get(1);
+                this.namespacePortion = parts.get(2);
+                this.localName = parts.get(3);
+                this.partitionIndex = getPartitionIndex(destination);
+                this.namespaceName = NamespaceName.get(property, cluster, namespacePortion);
+            } else {
                 throw new IllegalArgumentException("Invalid destination name: " + destination);
             }
 
-            this.property = parts.get(0);
-            this.cluster = parts.get(1);
-            this.namespacePortion = parts.get(2);
-            this.localName = parts.get(3);
-            this.partitionIndex = getPartitionIndex(destination);
 
-            NamespaceName.validateNamespaceName(property, cluster, namespacePortion);
-            if (checkNotNull(localName).isEmpty()) {
+            if (localName == null || localName.isEmpty()) {
                 throw new IllegalArgumentException("Invalid destination name: " + destination);
             }
         } catch (NullPointerException e) {
             throw new IllegalArgumentException("Invalid destination name: " + destination, e);
         }
-        namespaceName = NamespaceName.get(property, cluster, namespacePortion);
+
     }
 
     /**
@@ -156,6 +181,7 @@ public String getProperty() {
         return property;
     }
 
+    @Deprecated
     public String getCluster() {
         return cluster;
     }
@@ -229,9 +255,16 @@ public static int getPartitionIndex(String topic) {
      * @return the relative path to be used in persistence
      */
     public String getPersistenceNamingEncoding() {
-        // The convention is: domain://property/cluster/namespace/destination
-        // We want to persist in the order: property/cluster/namespace/domain/destination
-        return String.format("%s/%s/%s/%s/%s", property, cluster, namespacePortion, domain, getEncodedLocalName());
+        // The convention is: domain://property/namespace/topic
+        // We want to persist in the order: property/namespace/domain/topic
+
+        // For legacy naming scheme, the convention is: domain://property/cluster/namespace/topic
+        // We want to persist in the order: property/cluster/namespace/domain/topic
+        if (cluster == null) {
+            return String.format("%s/%s/%s/%s", property, namespacePortion, domain, getEncodedLocalName());
+        } else {
+            return String.format("%s/%s/%s/%s/%s", property, cluster, namespacePortion, domain, getEncodedLocalName());
+        }
     }
 
     /**
@@ -244,11 +277,15 @@ public String getPersistenceNamingEncoding() {
      * @return
      */
     public String getLookupName() {
-        return String.format("%s/%s/%s/%s/%s", domain, property, cluster, namespacePortion, getEncodedLocalName());
+        if (cluster == null) {
+            return String.format("%s/%s/%s/%s", domain, property, namespacePortion, getEncodedLocalName());
+        } else {
+            return String.format("%s/%s/%s/%s/%s", domain, property, cluster, namespacePortion, getEncodedLocalName());
+        }
     }
 
     public boolean isGlobal() {
-        return "global".equals(cluster);
+        return cluster == null || Constants.GLOBAL_CLUSTER.equalsIgnoreCase(cluster);
     }
 
     @Override
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
index a179f9af5..8ff474da8 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/NamespaceName.java
@@ -20,8 +20,6 @@
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
@@ -35,10 +33,10 @@
 
     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<String, NamespaceName> cache = CacheBuilder.newBuilder().maximumSize(100000)
             .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader<String, NamespaceName>() {
                 @Override
@@ -47,6 +45,11 @@ public NamespaceName load(String name) throws Exception {
                 }
             });
 
+    public static NamespaceName get(String property, String namespace) {
+        validateNamespaceName(property, namespace);
+        return get(property + '/' + namespace);
+    }
+
     public static NamespaceName get(String property, String cluster, String namespace) {
         validateNamespaceName(property, cluster, namespace);
         return get(property + '/' + cluster + '/' + namespace);
@@ -68,14 +71,37 @@ public static NamespaceName get(String namespace) {
     }
 
     private NamespaceName(String namespace) {
-        try {
-            checkNotNull(namespace);
-        } catch (NullPointerException e) {
+        if (namespace == null || namespace.isEmpty()) {
             throw new IllegalArgumentException("Invalid null namespace: " + namespace);
         }
 
         // Verify it's a proper namespace
-        validateNamespaceName(namespace);
+        // The namespace name is composed of <property>/<namespace>
+        // or in the legacy format with the cluster name:
+        // <property>/<cluster>/<namespace>
+        try {
+
+            String[] parts = namespace.split("/");
+            if (parts.length == 2) {
+                // New style namespace : <property>/<namespace>
+                validateNamespaceName(parts[0], parts[1]);
+
+                property = parts[0];
+                cluster = null;
+                localName = parts[1];
+            } else if (parts.length == 3) {
+                // Old style namespace: <property>/<cluster>/<namespace>
+                validateNamespaceName(parts[0], parts[1], parts[2]);
+
+                property = parts[0];
+                cluster = parts[1];
+                localName = parts[2];
+            } else {
+                throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace);
+            }
+        } catch (NullPointerException e) {
+            throw new IllegalArgumentException("Invalid namespace format. namespace: " + namespace, e);
+        }
         this.namespace = namespace;
     }
 
@@ -83,6 +109,7 @@ public String getProperty() {
         return property;
     }
 
+    @Deprecated
     public String getCluster() {
         return cluster;
     }
@@ -92,7 +119,7 @@ public String getLocalName() {
     }
 
     public boolean isGlobal() {
-        return "global".equals(cluster);
+        return cluster == null || Constants.GLOBAL_CLUSTER.equalsIgnoreCase(cluster);
     }
 
     public String getPersistentTopicName(String localTopic) {
@@ -136,46 +163,37 @@ public int hashCode() {
         return namespace.hashCode();
     }
 
-    public static void validateNamespaceName(String property, String cluster, String namespace) {
+    public static void validateNamespaceName(String property, String namespace) {
         try {
             checkNotNull(property);
-            checkNotNull(cluster);
             checkNotNull(namespace);
-            if (property.isEmpty() || cluster.isEmpty() || namespace.isEmpty()) {
+            if (property.isEmpty() || namespace.isEmpty()) {
                 throw new IllegalArgumentException(
-                        String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace));
+                        String.format("Invalid namespace format. namespace: %s/%s", property, namespace));
             }
             NamedEntity.checkName(property);
-            NamedEntity.checkName(cluster);
             NamedEntity.checkName(namespace);
         } catch (NullPointerException e) {
             throw new IllegalArgumentException(
-                    String.format("Invalid namespace format. namespace: %s/%s/%s", property, cluster, namespace), e);
+                    String.format("Invalid namespace format. namespace: %s/%s/%s", property, namespace), e);
         }
     }
 
-    private void validateNamespaceName(String namespace) {
-        // assume the namespace is in the form of <property>/<cluster>/<namespace>
+    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 74574d84e..caa16fad3 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/DestinationNameTest.java
@@ -118,13 +118,6 @@ void destination() {
             // Ok
         }
 
-        try {
-            DestinationName.get("persistent://property/cluster/namespace");
-            fail("Should have raised exception");
-        } catch (IllegalArgumentException e) {
-            // Ok
-        }
-
         try {
             DestinationName.get("property/cluster/namespace/destination");
             fail("Should have raised exception");
@@ -223,4 +216,28 @@ public void testDecodeEncode() throws Exception {
         assertEquals(name.getEncodedLocalName(), encodedName);
         assertEquals(name.getPersistenceNamingEncoding(), "prop/colo/ns/persistent/" + encodedName);
     }
+
+    @Test
+    public void testTopicNameWithoutCluster() throws Exception {
+        DestinationName dn = DestinationName.get("persistent://property/namespace/destination");
+
+        assertEquals(dn.getNamespace(), "property/namespace");
+
+        assertEquals(dn, DestinationName.get("persistent", "property", "namespace", "destination"));
+
+        assertEquals(dn.hashCode(),
+                DestinationName.get("persistent", "property", "namespace", "destination").hashCode());
+
+        assertEquals(dn.toString(), "persistent://property/namespace/destination");
+        assertEquals(dn.getDomain(), DestinationDomain.persistent);
+        assertEquals(dn.getProperty(), "property");
+        assertEquals(dn.getCluster(), null);
+        assertEquals(dn.getNamespacePortion(), "namespace");
+        assertEquals(dn.getNamespace(), "property/namespace");
+        assertEquals(dn.getLocalName(), "destination");
+
+        assertEquals(dn.getEncodedLocalName(), "destination");
+        assertEquals(dn.getPartitionedTopicName(), "persistent://property/namespace/destination");
+        assertEquals(dn.getPersistenceNamingEncoding(), "property/namespace/persistent/destination");
+    }
 }
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/NamespaceNameTest.java
index 7d6a7cf0b..8d13c2473 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");
+    }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services