pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] merlimat closed pull request #952: Reduce duplicate string allocation for GC improvements
Date Thu, 01 Jan 1970 00:00:00 GMT
merlimat closed pull request #952: Reduce duplicate string allocation for GC improvements
URL: https://github.com/apache/incubator-pulsar/pull/952
 
 
   

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/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
index 5beebc52f..2100cf4ff 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
@@ -231,7 +231,7 @@ protected Policies getNamespacePolicies(String property, String cluster, String
                     .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Namespace does not exist"));
             // fetch bundles from LocalZK-policies
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory()
-                    .getBundles(new NamespaceName(property, cluster, namespace));
+                    .getBundles(NamespaceName.get(property, cluster, namespace));
             BundlesData bundleData = NamespaceBundleFactory.getBundlesData(bundles);
             policies.bundles = bundleData != null ? bundleData : policies.bundles;
             return policies;
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/BrokerStats.java
index 45eefb6d3..8fdd31307 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/BrokerStats.java
@@ -173,7 +173,7 @@ public LoadManagerReport getLoadReport() throws Exception {
     public Map<Long, Collection<ResourceUnit>> getBrokerResourceAvailability(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) throws Exception {
         try {
-            NamespaceName ns = new NamespaceName(property, cluster, namespace);
+            NamespaceName ns = NamespaceName.get(property, cluster, namespace);
             LoadManager lm = pulsar().getLoadManager().get();
             if (lm instanceof SimpleLoadManagerImpl) {
                 return ((SimpleLoadManagerImpl) lm).getResourceAvailabilityFor(ns).asMap();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
index c16e5961a..b66165813 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/Namespaces.java
@@ -253,7 +253,7 @@ public void deleteNamespace(@PathParam("property") String property, @PathParam("
             @PathParam("namespace") String namespace,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
 
         validateAdminAccessOnProperty(property);
         validatePoliciesReadOnlyAccess();
@@ -359,7 +359,7 @@ public void deleteNamespace(@PathParam("property") String property, @PathParam("
     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 = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
 
         validateAdminAccessOnProperty(property);
         validatePoliciesReadOnlyAccess();
@@ -573,7 +573,7 @@ public void setNamespaceReplicationClusters(@PathParam("property") String proper
         }
 
         Entry<Policies, Stat> policiesNode = null;
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
 
         try {
             // Force to read the data s.t. the watch to the cache content is setup.
@@ -634,7 +634,7 @@ public void setNamespaceMessageTTL(@PathParam("property") String property, @Path
             throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for message TTL");
         }
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         Entry<Policies, Stat> policiesNode = null;
 
         try {
@@ -677,7 +677,7 @@ public void modifyDeduplication(@PathParam("property") String property, @PathPar
         validateAdminAccessOnProperty(property);
         validatePoliciesReadOnlyAccess();
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         Entry<Policies, Stat> policiesNode = null;
 
         try {
@@ -788,11 +788,11 @@ public void unloadNamespace(@PathParam("property") String property, @PathParam("
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
         Policies policies = getNamespacePolicies(property, cluster, namespace);
-        NamespaceName nsName = new NamespaceName(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++) {
@@ -827,10 +827,10 @@ public void unloadNamespaceBundle(@PathParam("property") String property, @PathP
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
         validatePoliciesReadOnlyAccess();
 
         if (!isBundleOwnedByAnyBroker(fqnn, policies.bundles, bundleRange)) {
@@ -868,10 +868,10 @@ public void splitNamespaceBundle(@PathParam("property") String property, @PathPa
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
         validatePoliciesReadOnlyAccess();
         NamespaceBundle nsBundle = validateNamespaceBundleOwnership(fqnn, policies.bundles, bundleRange, authoritative,
                 true);
@@ -900,7 +900,7 @@ public void setDispatchRate(@PathParam("property") String property, @PathParam("
         validateSuperUserAccess();
 
         Entry<Policies, Stat> policiesNode = null;
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
 
         try {
             final String path = path(POLICIES, property, cluster, namespace);
@@ -1231,7 +1231,7 @@ public void clearNamespaceBacklog(@PathParam("property") String property, @PathP
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
         validateAdminAccessOnProperty(property);
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         try {
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
             Exception exception = null;
@@ -1284,10 +1284,10 @@ public void clearNamespaceBundleBacklog(@PathParam("property") String property,
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership  for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
 
         clearBacklog(nsName, bundleRange, null);
@@ -1306,7 +1306,7 @@ public void clearNamespaceBacklogForSubscription(@PathParam("property") String p
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
         validateAdminAccessOnProperty(property);
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         try {
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
             Exception exception = null;
@@ -1359,10 +1359,10 @@ public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") St
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership  for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
 
         clearBacklog(nsName, bundleRange, subscription);
@@ -1380,7 +1380,7 @@ public void unsubscribeNamespace(@PathParam("property") String property, @PathPa
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
         validateAdminAccessOnProperty(property);
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         try {
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(nsName);
             Exception exception = null;
@@ -1432,10 +1432,10 @@ public void unsubscribeNamespaceBundle(@PathParam("property") String property, @
             validateClusterForProperty(property, cluster);
         } else {
             // check cluster ownership for a given global namespace: redirect if peer-cluster owns it
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
 
-        NamespaceName nsName = new NamespaceName(property, cluster, namespace);
+        NamespaceName nsName = NamespaceName.get(property, cluster, namespace);
         validateNamespaceBundleOwnership(nsName, policies.bundles, bundleRange, authoritative, true);
 
         unsubscribe(nsName, bundleRange, subscription);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java
index 9df72fac5..f4429a846 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/NonPersistentTopics.java
@@ -143,7 +143,7 @@ public void unloadTopic(@PathParam("property") String property, @PathParam("clus
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         unloadTopic(dn, authoritative);
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java
index 41415aa88..b7361b027 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/PersistentTopics.java
@@ -566,7 +566,7 @@ public void unloadTopic(@PathParam("property") String property, @PathParam("clus
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         unloadTopic(dn, authoritative);
     }
@@ -614,7 +614,7 @@ public void deleteTopic(@PathParam("property") String property, @PathParam("clus
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         } 
         List<String> subscriptions = Lists.newArrayList();
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
@@ -655,7 +655,7 @@ public PersistentTopicStats getStats(@PathParam("property") String property, @Pa
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         validateAdminAndClientPermission(dn);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         } 
         validateDestinationOwnership(dn, authoritative);
         Topic topic = getTopicReference(dn);
@@ -675,7 +675,7 @@ public PersistentTopicInternalStats getInternalStats(@PathParam("property") Stri
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         validateAdminAndClientPermission(dn);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         } 
         validateDestinationOwnership(dn, authoritative);
         Topic topic = getTopicReference(dn);
@@ -694,7 +694,7 @@ public void getManagedLedgerInfo(@PathParam("property") String property, @PathPa
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         validateAdminAccessOnProperty(dn.getProperty());
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         String managedLedger = dn.getPersistenceNamingEncoding();
         pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() {
@@ -729,7 +729,7 @@ public PartitionedTopicStats getPartitionedStats(@PathParam("property") String p
             throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found");
         }
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata);
         try {
@@ -758,7 +758,7 @@ public void deleteSubscription(@PathParam("property") String property, @PathPara
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -814,7 +814,7 @@ public void skipAllMessages(@PathParam("property") String property, @PathParam("
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -864,7 +864,7 @@ public void skipMessages(@PathParam("property") String property, @PathParam("clu
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -918,7 +918,7 @@ public void expireMessagesForAllSubscriptions(@PathParam("property") String prop
         final String destination = decode(destinationName);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -959,7 +959,7 @@ public void resetCursor(@PathParam("property") String property, @PathParam("clus
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -1037,7 +1037,7 @@ public void resetCursorOnPosition(@PathParam("property") String property, @PathP
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destination,
                 subName, messageId);
@@ -1089,7 +1089,7 @@ public Response peekNthMessage(@PathParam("property") String property, @PathPara
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -1184,7 +1184,7 @@ public PersistentOfflineTopicStats getBacklog(@PathParam("property") String prop
         destination = decode(destination);
         validateAdminAccessOnProperty(property);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         // Validate that namespace exists, throw 404 if it doesn't exist
         // note that we do not want to load the topic and hence skip validateAdminOperationOnDestination()
@@ -1235,7 +1235,7 @@ public MessageId terminate(@PathParam("property") String property, @PathParam("c
         destination = decode(destination);
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
@@ -1256,7 +1256,7 @@ public void expireMessages(String property, String cluster, String namespace, St
             int expireTimeInSeconds, boolean authoritative) {
         DestinationName dn = DestinationName.get(domain(), property, cluster, namespace, destination);
         if (cluster.equals(Namespaces.GLOBAL_CLUSTER)) {
-            validateGlobalNamespaceOwnership(new NamespaceName(property, cluster, namespace));
+            validateGlobalNamespaceOwnership(NamespaceName.get(property, cluster, namespace));
         }
         PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(property, cluster, namespace,
                 destination, authoritative);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java
index 4a8a25a6e..3eed02328 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/ResourceQuotas.java
@@ -95,7 +95,7 @@ public ResourceQuota getNamespaceBundleResourceQuota(@PathParam("property") Stri
             validateClusterForProperty(property, cluster);
         }
 
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
         NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
 
         try {
@@ -124,7 +124,7 @@ public void setNamespaceBundleResourceQuota(@PathParam("property") String proper
             validateClusterForProperty(property, cluster);
         }
 
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
         NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
 
         try {
@@ -160,7 +160,7 @@ public void removeNamespaceBundleResourceQuota(@PathParam("property") String pro
             validateClusterForProperty(property, cluster);
         }
 
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
         NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, policies.bundles, bundleRange);
 
         try {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTask.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTask.java
index 117430abc..d15ece0ec 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTask.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTask.java
@@ -91,7 +91,7 @@ public BundleSplitterTask(final PulsarService pulsar) {
                     final String namespace = LoadManagerShared.getNamespaceNameFromBundleName(bundle);
                     try {
                         final int bundleCount = pulsar.getNamespaceService()
-                                .getBundleCount(new NamespaceName(namespace));
+                                .getBundleCount(NamespaceName.get(namespace));
                         if (bundleCount < maxBundleCount) {
                             bundleCache.add(bundle);
                         } else {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
index 4cb985188..6f8b1d910 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
@@ -618,7 +618,7 @@ public void checkNamespaceBundleSplit() {
                     localData.getLastStats().remove(bundleName);
                     // Clear namespace bundle-cache
                     this.pulsar.getNamespaceService().getNamespaceBundleFactory()
-                            .invalidateBundleCache(new NamespaceName(namespaceName));
+                            .invalidateBundleCache(NamespaceName.get(namespaceName));
                     deleteBundleDataFromZookeeper(bundleName);
                     log.info("Successfully split namespace bundle {}", bundleName);
                 } catch (Exception e) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
index 43aef2c62..fadd0bdca 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
@@ -1283,7 +1283,7 @@ private boolean isLoadReportGenerationIntervalPassed() {
 
     // todo: changeme: this can be optimized, we don't have to iterate through everytime
     private boolean isBrokerAvailableForRebalancing(String bundleName, long maxLoadLevel) {
-        NamespaceName namespaceName = new NamespaceName(LoadManagerShared.getNamespaceNameFromBundleName(bundleName));
+        NamespaceName namespaceName = NamespaceName.get(LoadManagerShared.getNamespaceNameFromBundleName(bundleName));
         Map<Long, Set<ResourceUnit>> availableBrokers = sortedRankings.get();
         // this does not have "http://" in front, hacky but no time to pretty up
         Multimap<Long, ResourceUnit> brokers = getFinalCandidates(namespaceName, availableBrokers);
@@ -1427,7 +1427,7 @@ public void doNamespaceBundleSplit() throws Exception {
                 if (stats.topics <= 1) {
                     log.info("Unable to split hot namespace bundle {} since there is only one topic.", bundleName);
                 } else {
-                    NamespaceName namespaceName = new NamespaceName(
+                    NamespaceName namespaceName = NamespaceName.get(
                             LoadManagerShared.getNamespaceNameFromBundleName(bundleName));
                     int numBundles = pulsar.getNamespaceService().getBundleCount(namespaceName);
                     if (numBundles >= maxBundleCount) {
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 3ded848c1..d0fd7fda4 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
@@ -234,7 +234,7 @@ private boolean registerNamespace(String namespace, boolean ensureOwned) throws
         String myUrl = pulsar.getBrokerServiceUrl();
 
         try {
-            NamespaceName nsname = new NamespaceName(namespace);
+            NamespaceName nsname = NamespaceName.get(namespace);
 
             String otherUrl = null;
             NamespaceBundle nsFullBundle = null;
@@ -760,7 +760,7 @@ public void unloadSLANamespace() throws Exception {
 
         LOG.info("Checking owner for SLA namespace {}", namespaceName);
 
-        NamespaceBundle nsFullBundle = getFullBundle(new NamespaceName(namespaceName));
+        NamespaceBundle nsFullBundle = getFullBundle(NamespaceName.get(namespaceName));
         if (!getOwner(nsFullBundle).isPresent()) {
             // No one owns the namespace so no point trying to unload it
             // Next lookup will assign the bundle to this broker.
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 9ddeffa18..210b33507 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
@@ -78,7 +78,7 @@ public static final NamespaceBundle suBundleFromPath(String path, NamespaceBundl
         checkArgument(parts.length > 5);
 
         Range<Long> range = getHashRange(parts[5]);
-        return factory.getBundle(new NamespaceName(parts[2], parts[3], parts[4]), range);
+        return factory.getBundle(NamespaceName.get(parts[2], parts[3], parts[4]), range);
     }
 
     private static Range<Long> getHashRange(String rangePathPart) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
index 8aa18305e..1913dd5ff 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java
@@ -62,8 +62,8 @@ public AbstractReplicator(String topicName, String replicatorPrefix, String loca
         this.brokerService = brokerService;
         this.topicName = topicName;
         this.replicatorPrefix = replicatorPrefix;
-        this.localCluster = localCluster;
-        this.remoteCluster = remoteCluster;
+        this.localCluster = localCluster.intern();
+        this.remoteCluster = remoteCluster.intern();
         this.client = (PulsarClientImpl) brokerService.getReplicationClient(remoteCluster);
         this.producer = null;
         this.producerQueueSize = brokerService.pulsar().getConfiguration().getReplicationProducerQueueSize();
@@ -211,7 +211,7 @@ public static String getRemoteCluster(String remoteCursor) {
     }
 
     public static String getReplicatorName(String replicatorPrefix, String cluster) {
-        return String.format("%s.%s", replicatorPrefix, cluster);
+        return (replicatorPrefix + "." + cluster).intern();
     }
 
     private static final Logger log = LoggerFactory.getLogger(AbstractReplicator.class);
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 0ba8b762a..d85303bb4 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
@@ -946,7 +946,7 @@ public int getNumberOfNamespaceBundles() {
 
     @Override
     public void onUpdate(String path, Policies data, Stat stat) {
-        final NamespaceName namespace = new NamespaceName(NamespaceBundleFactory.getNamespaceFromPoliciesPath(path));
+        final NamespaceName namespace = NamespaceName.get(NamespaceBundleFactory.getNamespaceFromPoliciesPath(path));
 
         log.info("{} updating with {}", path, data);
 
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 383940a26..d6f6df29a 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
@@ -335,7 +335,7 @@ protected void handleConnect(CommandConnect connect) {
         remoteEndpointProtocolVersion = connect.getProtocolVersion();
         String version = connect.hasClientVersion() ? connect.getClientVersion() : null;
         if (isNotBlank(version) && !version.contains(" ") /* ignore default version: pulsar client */) {
-            this.clientVersion = version;
+            this.clientVersion = version.intern();
         }
     }
 
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 9c5566cdf..c22758dd1 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
@@ -330,7 +330,7 @@ protected static boolean isValidCluster(PulsarService pulsarSevice, String clust
      */
     protected void validateNamespaceOwnershipWithBundles(String property, String cluster, String namespace,
             boolean authoritative, boolean readOnly, BundlesData bundleData) {
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
 
         try {
             NamespaceBundles bundles = pulsar().getNamespaceService().getNamespaceBundleFactory().getBundles(fqnn,
@@ -349,7 +349,7 @@ protected void validateNamespaceOwnershipWithBundles(String property, String clu
 
     protected void validateBundleOwnership(String property, String cluster, String namespace, boolean authoritative,
             boolean readOnly, NamespaceBundle bundle) {
-        NamespaceName fqnn = new NamespaceName(property, cluster, namespace);
+        NamespaceName fqnn = NamespaceName.get(property, cluster, namespace);
 
         try {
             validateBundleOwnership(bundle, authoritative, readOnly);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java
index 996a245a7..44dd68670 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java
@@ -95,7 +95,7 @@ public NamespaceBundleFactory(PulsarService pulsar, HashFunction hashFunc) {
                 .registerListener((String path, LocalPolicies data, Stat stat) -> {
                     String[] paths = path.split(LOCAL_POLICIES_ROOT + "/");
                     if (paths.length == 2) {
-                        invalidateBundleCache(new NamespaceName(paths[1]));
+                        invalidateBundleCache(NamespaceName.get(paths[1]));
                     }
                 });
 
@@ -108,7 +108,7 @@ public NamespaceBundleFactory(PulsarService pulsar, HashFunction hashFunc) {
 
     @Override
     public void onUpdate(String path, LocalPolicies data, Stat stat) {
-        final NamespaceName namespace = new NamespaceName(getNamespaceFromPoliciesPath(path));
+        final NamespaceName namespace = NamespaceName.get(getNamespaceFromPoliciesPath(path));
 
         try {
             LOG.info("Policy updated for namespace {}, refreshing the bundle cache.", namespace);
@@ -155,7 +155,7 @@ public NamespaceBundle getBundle(String namespace, String bundleRange) {
         Long upperEndpoint = Long.decode(boundaries[1]);
         Range<Long> hashRange = Range.range(lowerEndpoint, BoundType.CLOSED, upperEndpoint,
                 (upperEndpoint.equals(NamespaceBundles.FULL_UPPER_BOUND)) ? BoundType.CLOSED : BoundType.OPEN);
-        return getBundle(new NamespaceName(namespace), hashRange);
+        return getBundle(NamespaceName.get(namespace), hashRange);
     }
     
     public NamespaceBundle getFullBundle(NamespaceName fqnn) throws Exception {
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 0ef5d9c90..7736c42b3 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
@@ -611,7 +611,7 @@ public void namespaces() throws PulsarAdminException, PulsarServerException, Exc
         admin.persistentTopics().delete("persistent://prop-xyz/use/ns1/my-topic");
 
         admin.namespaces().unloadNamespaceBundle("prop-xyz/use/ns1", "0x00000000_0xffffffff");
-        NamespaceName ns = new NamespaceName("prop-xyz/use/ns1");
+        NamespaceName ns = NamespaceName.get("prop-xyz/use/ns1");
         // Now, w/ bundle policies, we will use default bundle
         NamespaceBundle defaultBundle = bundleFactory.getFullBundle(ns);
         int i = 0;
@@ -900,7 +900,7 @@ public void testNamespaceSplitBundle() throws Exception {
         }
 
         // bundle-factory cache must have updated split bundles
-        NamespaceBundles bundles = bundleFactory.getBundles(new NamespaceName(namespace));
+        NamespaceBundles bundles = bundleFactory.getBundles(NamespaceName.get(namespace));
         String[] splitRange = { namespace + "/0x00000000_0x7fffffff", namespace + "/0x7fffffff_0xffffffff" };
         for (int i = 0; i < bundles.getBundles().size(); i++) {
             assertEquals(bundles.getBundles().get(i).toString(), splitRange[i]);
@@ -942,7 +942,7 @@ public void testNamespaceUnloadBundle() throws Exception {
         }
 
         // check that no one owns the namespace
-        NamespaceBundle bundle = bundleFactory.getBundle(new NamespaceName("prop-xyz/use/ns1"),
+        NamespaceBundle bundle = bundleFactory.getBundle(NamespaceName.get("prop-xyz/use/ns1"),
                 Range.range(0L, BoundType.CLOSED, 0xffffffffL, BoundType.CLOSED));
         assertFalse(pulsar.getNamespaceService().isServiceUnitOwned(bundle));
         assertFalse(otherPulsar.getNamespaceService().isServiceUnitOwned(bundle));
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 0158722ca..0c384ebdd 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
@@ -105,11 +105,11 @@ public void initNamespace() throws Exception {
         testLocalNamespaces = Lists.newArrayList();
         testGlobalNamespaces = Lists.newArrayList();
 
-        testLocalNamespaces.add(new NamespaceName(this.testProperty, this.testLocalCluster, "test-namespace-1"));
-        testLocalNamespaces.add(new NamespaceName(this.testProperty, this.testLocalCluster, "test-namespace-2"));
-        testLocalNamespaces.add(new NamespaceName(this.testProperty, this.testOtherCluster, "test-other-namespace-1"));
+        testLocalNamespaces.add(NamespaceName.get(this.testProperty, this.testLocalCluster, "test-namespace-1"));
+        testLocalNamespaces.add(NamespaceName.get(this.testProperty, this.testLocalCluster, "test-namespace-2"));
+        testLocalNamespaces.add(NamespaceName.get(this.testProperty, this.testOtherCluster, "test-other-namespace-1"));
 
-        testGlobalNamespaces.add(new NamespaceName(this.testProperty, "global", "test-global-ns1"));
+        testGlobalNamespaces.add(NamespaceName.get(this.testProperty, "global", "test-global-ns1"));
 
         uriField = PulsarWebResource.class.getDeclaredField("uri");
         uriField.setAccessible(true);
@@ -164,9 +164,9 @@ public void testCreateNamespaces() throws Exception {
         }
 
         List<NamespaceName> nsnames = Lists.newArrayList();
-        nsnames.add(new NamespaceName("my-property", "use", "create-namespace-1"));
-        nsnames.add(new NamespaceName("my-property", "use", "create-namespace-2"));
-        nsnames.add(new NamespaceName("my-property", "usc", "create-other-namespace-1"));
+        nsnames.add(NamespaceName.get("my-property", "use", "create-namespace-1"));
+        nsnames.add(NamespaceName.get("my-property", "use", "create-namespace-2"));
+        nsnames.add(NamespaceName.get("my-property", "usc", "create-other-namespace-1"));
         createTestNamespaces("my-property", nsnames, new BundlesData());
 
         try {
@@ -652,7 +652,7 @@ public void testDeleteNamespaceWithBundles() throws Exception {
         String bundledNsLocal = "test-bundled-namespace-1";
         BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff"));
         createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-        final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+        final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
 
         org.apache.pulsar.client.admin.Namespaces namespacesAdmin = mock(org.apache.pulsar.client.admin.Namespaces.class);
         doReturn(namespacesAdmin).when(admin).namespaces();
@@ -835,7 +835,7 @@ public void testSplitBundles() throws Exception {
         String bundledNsLocal = "test-bundled-namespace-1";
         BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff"));
         createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-        final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+        final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
 
         OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache());
         doNothing().when(MockOwnershipCache).disableOwnership(any(NamespaceBundle.class));
@@ -867,7 +867,7 @@ public void testSplitBundleWithUnDividedRange() throws Exception {
         BundlesData bundleData = new BundlesData(
                 Lists.newArrayList("0x00000000", "0x08375b1a", "0x08375b1b", "0xffffffff"));
         createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-        final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+        final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
 
         OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache());
         doNothing().when(MockOwnershipCache).disableOwnership(any(NamespaceBundle.class));
@@ -891,7 +891,7 @@ public void testUnloadNamespaceWithBundles() throws Exception {
         String bundledNsLocal = "test-bundled-namespace-1";
         BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0x80000000", "0xffffffff"));
         createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-        final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+        final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
 
         doReturn(Optional.of(localWebServiceUrl)).when(nsSvc)
                 .getWebServiceUrl(Mockito.argThat(new Matcher<NamespaceBundle>() {
@@ -1005,7 +1005,7 @@ public void testValidateNamespaceOwnershipWithBundles() throws Exception {
             String bundledNsLocal = "test-bundled-namespace-1";
             BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff"));
             createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-            final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+            final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
             mockWebUrl(localWebServiceUrl, testNs);
 
             OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache());
@@ -1027,7 +1027,7 @@ public void testRetention() throws Exception {
             String bundledNsLocal = "test-bundled-namespace-1";
             BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff"));
             createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-            final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+            final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
             mockWebUrl(localWebServiceUrl, testNs);
 
             OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache());
@@ -1052,7 +1052,7 @@ public void testValidateDestinationOwnership() throws Exception {
             String bundledNsLocal = "test-bundled-namespace-1";
             BundlesData bundleData = new BundlesData(Lists.newArrayList("0x00000000", "0xffffffff"));
             createBundledTestNamespaces(this.testProperty, this.testLocalCluster, bundledNsLocal, bundleData);
-            final NamespaceName testNs = new NamespaceName(this.testProperty, this.testLocalCluster, bundledNsLocal);
+            final NamespaceName testNs = NamespaceName.get(this.testProperty, this.testLocalCluster, bundledNsLocal);
             OwnershipCache MockOwnershipCache = spy(pulsar.getNamespaceService().getOwnershipCache());
             doNothing().when(MockOwnershipCache).disableOwnership(any(NamespaceBundle.class));
             Field ownership = NamespaceService.class.getDeclaredField("ownershipCache");
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/ResourceQuotaCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/ResourceQuotaCacheTest.java
index 860b9f10e..8a620a46d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/ResourceQuotaCacheTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/ResourceQuotaCacheTest.java
@@ -102,7 +102,7 @@ public void testGetSetDefaultQuota() throws Exception {
     @Test
     public void testGetSetBundleQuota() throws Exception {
         ResourceQuotaCache cache = new ResourceQuotaCache(zkCache);
-        NamespaceBundle testBundle = bundleFactory.getFullBundle(new NamespaceName("pulsar/test/ns-2"));
+        NamespaceBundle testBundle = bundleFactory.getFullBundle(NamespaceName.get("pulsar/test/ns-2"));
         ResourceQuota quota1 = ResourceQuotaCache.getInitialQuotaValue();
         ResourceQuota quota2 = new ResourceQuota();
         quota2.setMsgRateIn(10);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
index 47ecd69ef..8511e4058 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
@@ -829,7 +829,7 @@ public void testGetLeastLoadedBasic() throws Exception {
         sortedRankings.set(loadManager, sortedRankingsInstance);
 
         ResourceUnit found = ((SimpleLoadManagerImpl) loadManager)
-                .getLeastLoaded(new NamespaceName("pulsar/use/primary-ns.10"));
+                .getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10"));
         assertEquals("http://prod1-broker1.messaging.use.example.com:8080", found.getResourceId());
 
         zkCacheField.set(pulsarServices[0], originalLZK1);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTestingUtils.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTestingUtils.java
index 6c0870581..93e57b474 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTestingUtils.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTestingUtils.java
@@ -30,7 +30,7 @@
     public static NamespaceBundle[] makeBundles(final NamespaceBundleFactory nsFactory, final String property,
             final String cluster, final String namespace, final int numBundles) {
         final NamespaceBundle[] result = new NamespaceBundle[numBundles];
-        final NamespaceName namespaceName = new NamespaceName(property, cluster, namespace);
+        final NamespaceName namespaceName = NamespaceName.get(property, cluster, namespace);
         for (int i = 0; i < numBundles - 1; ++i) {
             final long lower = NamespaceBundles.FULL_UPPER_BOUND * i / numBundles;
             final long upper = NamespaceBundles.FULL_UPPER_BOUND * (i + 1) / numBundles;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java
index 81c22e07f..f3d7d11e7 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerImplTest.java
@@ -202,7 +202,7 @@ void shutdown() throws Exception {
     }
 
     private NamespaceBundle makeBundle(final String property, final String cluster, final String namespace) {
-        return nsFactory.getBundle(new NamespaceName(property, cluster, namespace),
+        return nsFactory.getBundle(NamespaceName.get(property, cluster, namespace),
                 Range.range(NamespaceBundles.FULL_LOWER_BOUND, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND,
                         BoundType.CLOSED));
     }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java
index 2a0d9d175..8f367e484 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java
@@ -235,7 +235,7 @@ public void testBasicBrokerSelection() throws Exception {
         sortedRankings.set(loadManager, sortedRankingsInstance);
 
         ResourceUnit found = ((SimpleLoadManagerImpl) loadManager)
-                .getLeastLoaded(new NamespaceName("pulsar/use/primary-ns.10"));
+                .getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10"));
         // broker is not active so found should be null
         assertEquals(found, null, "found a broker when expected none to be found");
 
@@ -282,7 +282,7 @@ public void testPrimary() throws Exception {
         setObjectField(SimpleLoadManagerImpl.class, loadManager, "sortedRankings", sortedRankingsInstance);
 
         ResourceUnit found = ((SimpleLoadManagerImpl) loadManager)
-                .getLeastLoaded(new NamespaceName("pulsar/use/primary-ns.10"));
+                .getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10"));
         // broker is not active so found should be null
         assertNotEquals(found, null, "did not find a broker when expected one to be found");
 
@@ -338,7 +338,7 @@ public void testPrimarySecondary() throws Exception {
         sortedRankings.set(loadManager, sortedRankingsInstance);
 
         ResourceUnit found = ((SimpleLoadManagerImpl) loadManager)
-                .getLeastLoaded(new NamespaceName("pulsar/use/primary-ns.10"));
+                .getLeastLoaded(NamespaceName.get("pulsar/use/primary-ns.10"));
         assertEquals(found.getResourceId(), ru1.getResourceId());
 
         zkCacheField.set(pulsar1, originalLZK1);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
index aa0979b04..332730733 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
@@ -106,7 +106,7 @@ public void testSplitAndOwnBundles() throws Exception {
         ownership.setAccessible(true);
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
         NamespaceService namespaceService = pulsar.getNamespaceService();
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
         NamespaceBundle originalBundle = bundles.findBundle(dn);
@@ -176,7 +176,7 @@ public void testSplitMapWithRefreshedStatMap() throws Exception {
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
 
         NamespaceService namespaceService = pulsar.getNamespaceService();
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
         NamespaceBundle originalBundle = bundles.findBundle(dn);
@@ -229,7 +229,7 @@ public void testIsServiceUnitDisabled() throws Exception {
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
 
         NamespaceService namespaceService = pulsar.getNamespaceService();
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
         NamespaceBundle originalBundle = bundles.findBundle(dn);
@@ -252,7 +252,7 @@ public void testremoveOwnershipNamespaceBundle() throws Exception {
         ownership.set(pulsar.getNamespaceService(), ownershipCache);
 
         NamespaceService namespaceService = pulsar.getNamespaceService();
-        NamespaceName nsname = new NamespaceName("prop/use/ns1");
+        NamespaceName nsname = NamespaceName.get("prop/use/ns1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
 
         NamespaceBundle bundle = bundles.getBundles().get(0);
@@ -341,7 +341,7 @@ public void testCreateNamespaceWithDefaultNumberOfBundles() throws Exception {
         ownership.setAccessible(true);
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
         NamespaceService namespaceService = pulsar.getNamespaceService();
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
         NamespaceBundle originalBundle = bundles.findBundle(dn);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java
index dd4879e31..02128e879 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java
@@ -124,7 +124,7 @@ public void testConstructor() {
     public void testDisableOwnership() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
 
-        NamespaceBundle testBundle = bundleFactory.getFullBundle(new NamespaceName("pulsar/test/ns-1"));
+        NamespaceBundle testBundle = bundleFactory.getFullBundle(NamespaceName.get("pulsar/test/ns-1"));
         assertFalse(cache.getOwnerAsync(testBundle).get().isPresent());
 
         NamespaceEphemeralData data1 = cache.tryAcquiringOwnership(testBundle).get();
@@ -139,7 +139,7 @@ public void testDisableOwnership() throws Exception {
     @Test
     public void testGetOrSetOwner() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
-        NamespaceBundle testFullBundle = bundleFactory.getFullBundle(new NamespaceName("pulsar/test/ns-2"));
+        NamespaceBundle testFullBundle = bundleFactory.getFullBundle(NamespaceName.get("pulsar/test/ns-2"));
         // case 1: no one owns the namespace
         assertFalse(cache.getOwnerAsync(testFullBundle).get().isPresent());
 
@@ -170,7 +170,7 @@ public void testGetOrSetOwner() throws Exception {
     @Test
     public void testGetOwner() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
-        NamespaceBundle testBundle = bundleFactory.getFullBundle(new NamespaceName("pulsar/test/ns-3"));
+        NamespaceBundle testBundle = bundleFactory.getFullBundle(NamespaceName.get("pulsar/test/ns-3"));
         // case 1: no one owns the namespace
         assertFalse(cache.getOwnerAsync(testBundle).get().isPresent());
         // case 2: someone owns the namespace
@@ -191,14 +191,14 @@ public void testGetOwner() throws Exception {
         MockZooKeeper mockZk = (MockZooKeeper) zkCache.getZooKeeper();
         mockZk.failNow(KeeperException.Code.NONODE);
         Optional<NamespaceEphemeralData> res = cache
-                .getOwnerAsync(bundleFactory.getFullBundle(new NamespaceName("pulsar/test/ns-none"))).get();
+                .getOwnerAsync(bundleFactory.getFullBundle(NamespaceName.get("pulsar/test/ns-none"))).get();
         assertFalse(res.isPresent());
     }
 
     @Test
     public void testGetOwnedServiceUnit() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
-        NamespaceName testNs = new NamespaceName("pulsar/test/ns-5");
+        NamespaceName testNs = NamespaceName.get("pulsar/test/ns-5");
         NamespaceBundle testBundle = bundleFactory.getFullBundle(testNs);
         // case 1: no one owns the namespace
         assertFalse(cache.getOwnerAsync(testBundle).get().isPresent());
@@ -247,7 +247,7 @@ public void testGetOwnedServiceUnit() throws Exception {
     @Test
     public void testGetOwnedServiceUnits() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
-        NamespaceName testNs = new NamespaceName("pulsar/test/ns-6");
+        NamespaceName testNs = NamespaceName.get("pulsar/test/ns-6");
         NamespaceBundle testBundle = bundleFactory.getFullBundle(testNs);
         // case 1: no one owns the namespace
         assertFalse(cache.getOwnerAsync(testBundle).get().isPresent());
@@ -282,7 +282,7 @@ public void testGetOwnedServiceUnits() throws Exception {
     @Test
     public void testRemoveOwnership() throws Exception {
         OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory);
-        NamespaceName testNs = new NamespaceName("pulsar/test/ns-7");
+        NamespaceName testNs = NamespaceName.get("pulsar/test/ns-7");
         NamespaceBundle bundle = bundleFactory.getFullBundle(testNs);
         // case 1: no one owns the namespace
         assertFalse(cache.getOwnerAsync(bundle).get().isPresent());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java
index 42b75f02e..f25c85ebc 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java
@@ -232,10 +232,10 @@ public void testConfigChangeNegativeCases() throws Exception {
         Assert.assertNotNull(pulsar1, "pulsar1 is null");
         Assert.assertNotNull(pulsar1.getNamespaceService(), "pulsar1.getNamespaceService() is null");
         NamespaceBundle globalNsBundle = pulsar1.getNamespaceService().getNamespaceBundleFactory()
-                .getFullBundle(new NamespaceName("pulsar/global/ns"));
+                .getFullBundle(NamespaceName.get("pulsar/global/ns"));
         ownerCache.tryAcquiringOwnership(globalNsBundle);
         Assert.assertNotNull(ownerCache.getOwnedBundle(globalNsBundle),
-                "pulsar1.getNamespaceService().getOwnedServiceUnit(new NamespaceName(\"pulsar/global/ns\")) is null");
+                "pulsar1.getNamespaceService().getOwnedServiceUnit(NamespaceName.get(\"pulsar/global/ns\")) is null");
         Field stateField = OwnedBundle.class.getDeclaredField("isActive");
         stateField.setAccessible(true);
         // set the namespace to be disabled
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
index 7b3261fce..e86f51676 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
@@ -55,14 +55,14 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar.old.ns"), null, null);
+            new NamespaceBundle(NamespaceName.get("pulsar.old.ns"), null, null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
             // OK, expected
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                     Range.range(0L, BoundType.CLOSED, 0L, BoundType.OPEN), null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
@@ -70,7 +70,7 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"), Range.range(0L, BoundType.OPEN, 1L, BoundType.OPEN),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"), Range.range(0L, BoundType.OPEN, 1L, BoundType.OPEN),
                     null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
@@ -78,7 +78,7 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                     Range.range(1L, BoundType.CLOSED, 1L, BoundType.OPEN), null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
@@ -86,7 +86,7 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                     Range.range(0L, BoundType.CLOSED, 1L, BoundType.CLOSED), null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
@@ -94,7 +94,7 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                     Range.range(0L, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.OPEN), null);
             fail("Should have failed w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
@@ -102,14 +102,14 @@ public void testConstructor() {
         }
 
         try {
-            new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+            new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                     Range.range(0L, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.CLOSED), null);
             fail("Should have failed w/ null pointer exception");
         } catch (NullPointerException npe) {
             // OK, expected
         }
 
-        NamespaceBundle bundle = new NamespaceBundle(new NamespaceName("pulsar/use/ns"),
+        NamespaceBundle bundle = new NamespaceBundle(NamespaceName.get("pulsar/use/ns"),
                 Range.range(0L, BoundType.CLOSED, 1L, BoundType.OPEN), factory);
         assertTrue(bundle.getKeyRange().lowerEndpoint().equals(0L));
         assertEquals(bundle.getKeyRange().lowerBoundType(), BoundType.CLOSED);
@@ -130,10 +130,10 @@ private NamespaceBundleFactory getNamespaceBundleFactory() {
 
     @Test
     public void testGetBundle() throws Exception {
-        NamespaceBundle bundle = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0L, BoundType.CLOSED, 0xffffffffL, BoundType.CLOSED));
         assertNotNull(bundle);
-        NamespaceBundle bundle2 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle2 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0L, BoundType.CLOSED, 0xffffffffL, BoundType.CLOSED));
         // Don't call equals and make sure those two are the same instance
         assertEquals(bundle, bundle2);
@@ -142,9 +142,9 @@ public void testGetBundle() throws Exception {
 
     @Test
     public void testCompareTo() throws Exception {
-        NamespaceBundle bundle = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
-        NamespaceBundle bundle2 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle2 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0x20000000l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         try {
             bundle.compareTo(bundle2);
@@ -153,36 +153,36 @@ public void testCompareTo() throws Exception {
             // OK, expected
         }
 
-        NamespaceBundle bundle0 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle0 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x10000000L, BoundType.OPEN));
         assertTrue(bundle0.compareTo(bundle2) < 0);
         assertTrue(bundle2.compareTo(bundle0) > 0);
-        NamespaceBundle bundle1 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle1 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x20000000L, BoundType.OPEN));
         assertTrue(bundle1.compareTo(bundle2) < 0);
 
-        NamespaceBundle bundle3 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle3 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         assertTrue(bundle.compareTo(bundle3) == 0);
 
-        NamespaceBundle otherBundle = factory.getBundle(new NamespaceName("pulsar/use/ns2"),
+        NamespaceBundle otherBundle = factory.getBundle(NamespaceName.get("pulsar/use/ns2"),
                 Range.range(0x10000000l, BoundType.CLOSED, 0x30000000L, BoundType.OPEN));
         assertTrue(otherBundle.compareTo(bundle0) > 0);
     }
 
     @Test
     public void testEquals() throws Exception {
-        NamespaceBundle bundle = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
-        NamespaceBundle bundle2 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle2 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0x20000000l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         assertTrue(!bundle.equals(bundle2));
 
-        NamespaceBundle bundle0 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle0 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         assertTrue(bundle0.equals(bundle));
 
-        NamespaceBundle otherBundle = factory.getBundle(new NamespaceName("pulsar/use/ns2"),
+        NamespaceBundle otherBundle = factory.getBundle(NamespaceName.get("pulsar/use/ns2"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         assertTrue(!otherBundle.equals(bundle));
     }
@@ -196,21 +196,21 @@ public void testIncludes() throws Exception {
         NamespaceBundle bundle = factory.getBundle(dn.getNamespaceObject(),
                 Range.range(hashKey / 2, BoundType.CLOSED, upper, upperType));
         assertTrue(bundle.includes(dn));
-        bundle = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(upper, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.CLOSED));
         assertTrue(!bundle.includes(dn));
 
-        NamespaceBundle otherBundle = factory.getBundle(new NamespaceName("pulsar/use/ns2"),
+        NamespaceBundle otherBundle = factory.getBundle(NamespaceName.get("pulsar/use/ns2"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
         assertTrue(!otherBundle.includes(dn));
     }
 
     @Test
     public void testToString() throws Exception {
-        NamespaceBundle bundle0 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        NamespaceBundle bundle0 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0l, BoundType.CLOSED, 0x10000000L, BoundType.OPEN));
         assertEquals(bundle0.toString(), "pulsar/use/ns1/0x00000000_0x10000000");
-        bundle0 = factory.getBundle(new NamespaceName("pulsar/use/ns1"),
+        bundle0 = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(0x10000000l, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.CLOSED));
         assertEquals(bundle0.toString(), "pulsar/use/ns1/0x10000000_0xffffffff");
     }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
index d56984eb5..4c2bebe16 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
@@ -69,21 +69,21 @@ public void testConstructor() throws Exception {
         }
 
         try {
-            new NamespaceBundles(new NamespaceName("pulsar/use/ns2"), (SortedSet<Long>) null, null);
+            new NamespaceBundles(NamespaceName.get("pulsar/use/ns2"), (SortedSet<Long>) null, null);
             fail("Should fail w/ null pointer exception");
         } catch (NullPointerException npe) {
             // OK, expected
         }
 
         try {
-            new NamespaceBundles(new NamespaceName("pulsar.use.ns2"), (SortedSet<Long>) null, null);
+            new NamespaceBundles(NamespaceName.get("pulsar.use.ns2"), (SortedSet<Long>) null, null);
             fail("Should fail w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
             // OK, expected
         }
 
         try {
-            new NamespaceBundles(new NamespaceName("pulsar/use/ns2"), (SortedSet<Long>) null, factory);
+            new NamespaceBundles(NamespaceName.get("pulsar/use/ns2"), (SortedSet<Long>) null, factory);
             fail("Should fail w/ null pointer exception");
         } catch (NullPointerException npe) {
             // OK, expected
@@ -91,7 +91,7 @@ public void testConstructor() throws Exception {
 
         SortedSet<Long> partitions = Sets.newTreeSet();
         try {
-            new NamespaceBundles(new NamespaceName("pulsar/use/ns2"), partitions, factory);
+            new NamespaceBundles(NamespaceName.get("pulsar/use/ns2"), partitions, factory);
             fail("Should fail w/ illegal argument exception");
         } catch (IllegalArgumentException iae) {
             // OK, expected
@@ -101,7 +101,7 @@ public void testConstructor() throws Exception {
         partitions.add(0x10000000l);
         partitions.add(0x40000000l);
         partitions.add(0xffffffffl);
-        NamespaceBundles bundles = new NamespaceBundles(new NamespaceName("pulsar/use/ns2"), partitions, factory);
+        NamespaceBundles bundles = new NamespaceBundles(NamespaceName.get("pulsar/use/ns2"), partitions, factory);
         Field partitionField = NamespaceBundles.class.getDeclaredField("partitions");
         Field nsField = NamespaceBundles.class.getDeclaredField("nsname");
         Field bundlesField = NamespaceBundles.class.getDeclaredField("bundles");
@@ -142,7 +142,7 @@ public void testFindBundle() throws Exception {
         partitions.add(0xb0000000l);
         partitions.add(0xc0000000l);
         partitions.add(0xffffffffl);
-        NamespaceBundles bundles = new NamespaceBundles(new NamespaceName("pulsar/global/ns1"), partitions, factory);
+        NamespaceBundles bundles = new NamespaceBundles(NamespaceName.get("pulsar/global/ns1"), partitions, factory);
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundle bundle = bundles.findBundle(dn);
         assertTrue(bundle.includes(dn));
@@ -175,7 +175,7 @@ public void testFindBundle() throws Exception {
 
     @Test
     public void testsplitBundles() throws Exception {
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = factory.getBundles(nsname);
         NamespaceBundle bundle = bundles.findBundle(dn);
@@ -219,7 +219,7 @@ public void testsplitBundles() throws Exception {
     @Test
     public void testSplitBundleInTwo() throws Exception {
         final int NO_BUNDLES = 2;
-        NamespaceName nsname = new NamespaceName("pulsar/global/ns1");
+        NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
         DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = factory.getBundles(nsname);
         NamespaceBundle bundle = bundles.findBundle(dn);
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
index 35beb0a7d..9b7eac68e 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
@@ -73,7 +73,7 @@ public NamespacesImpl(WebTarget web, Authentication auth) {
     @Override
     public List<String> getDestinations(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
                     .path("destinations")).get(new GenericType<List<String>>() {
                     });
@@ -85,7 +85,7 @@ public NamespacesImpl(WebTarget web, Authentication auth) {
     @Override
     public Policies getPolicies(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()))
                     .get(Policies.class);
         } catch (Exception e) {
@@ -101,7 +101,7 @@ public void createNamespace(String namespace, int numBundles) throws PulsarAdmin
     @Override
     public void createNamespace(String namespace, BundlesData bundlesData) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()))
                     .put(Entity.entity(bundlesData, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -112,7 +112,7 @@ public void createNamespace(String namespace, BundlesData bundlesData) throws Pu
     @Override
     public void createNamespace(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()))
                     .put(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -123,7 +123,7 @@ public void createNamespace(String namespace) throws PulsarAdminException {
     @Override
     public void deleteNamespace(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()))
                     .delete(ErrorData.class);
         } catch (Exception e) {
@@ -134,7 +134,7 @@ public void deleteNamespace(String namespace) throws PulsarAdminException {
     @Override
     public void deleteNamespaceBundle(String namespace, String bundleRange) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundleRange))
                     .delete(ErrorData.class);
         } catch (Exception e) {
@@ -145,7 +145,7 @@ public void deleteNamespaceBundle(String namespace, String bundleRange) throws P
     @Override
     public Map<String, Set<AuthAction>> getPermissions(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("permissions"))
                             .get(new GenericType<Map<String, Set<AuthAction>>>() {
@@ -159,7 +159,7 @@ public void deleteNamespaceBundle(String namespace, String bundleRange) throws P
     public void grantPermissionOnNamespace(String namespace, String role, Set<AuthAction> actions)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("permissions")
                     .path(role)).post(Entity.entity(actions, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -170,7 +170,7 @@ public void grantPermissionOnNamespace(String namespace, String role, Set<AuthAc
     @Override
     public void revokePermissionsOnNamespace(String namespace, String role) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("permissions")
                     .path(role)).delete(ErrorData.class);
         } catch (Exception e) {
@@ -181,7 +181,7 @@ public void revokePermissionsOnNamespace(String namespace, String role) throws P
     @Override
     public List<String> getNamespaceReplicationClusters(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("replication"))
                             .get(new GenericType<List<String>>() {
@@ -194,7 +194,7 @@ public void revokePermissionsOnNamespace(String namespace, String role) throws P
     @Override
     public void setNamespaceReplicationClusters(String namespace, List<String> clusterIds) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("replication"))
                     .post(Entity.entity(clusterIds, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -205,7 +205,7 @@ public void setNamespaceReplicationClusters(String namespace, List<String> clust
     @Override
     public int getNamespaceMessageTTL(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("messageTTL"))
                             .get(new GenericType<Integer>() {
@@ -219,7 +219,7 @@ public int getNamespaceMessageTTL(String namespace) throws PulsarAdminException
     @Override
     public void setNamespaceMessageTTL(String namespace, int ttlInSeconds) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("messageTTL"))
                     .post(Entity.entity(ttlInSeconds, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -230,7 +230,7 @@ public void setNamespaceMessageTTL(String namespace, int ttlInSeconds) throws Pu
     @Override
     public void setDeduplicationStatus(String namespace, boolean enableDeduplication) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("deduplication"))
                     .post(Entity.entity(enableDeduplication, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -241,7 +241,7 @@ public void setDeduplicationStatus(String namespace, boolean enableDeduplication
     @Override
     public Map<BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
                     .path("backlogQuotaMap")).get(new GenericType<Map<BacklogQuotaType, BacklogQuota>>() {
                     });
@@ -253,7 +253,7 @@ public void setDeduplicationStatus(String namespace, boolean enableDeduplication
     @Override
     public void setBacklogQuota(String namespace, BacklogQuota backlogQuota) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
                     .path("backlogQuota")).post(Entity.entity(backlogQuota, MediaType.APPLICATION_JSON),
                             ErrorData.class);
@@ -265,7 +265,7 @@ public void setBacklogQuota(String namespace, BacklogQuota backlogQuota) throws
     @Override
     public void removeBacklogQuota(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("backlogQuota")
                     .queryParam("backlogQuotaType", BacklogQuotaType.destination_storage.toString()))
                             .delete(ErrorData.class);
@@ -277,7 +277,7 @@ public void removeBacklogQuota(String namespace) throws PulsarAdminException {
     @Override
     public void setPersistence(String namespace, PersistencePolicies persistence) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("persistence"))
                     .post(Entity.entity(persistence, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -288,7 +288,7 @@ public void setPersistence(String namespace, PersistencePolicies persistence) th
     @Override
     public PersistencePolicies getPersistence(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("persistence"))
                             .get(PersistencePolicies.class);
@@ -300,7 +300,7 @@ public PersistencePolicies getPersistence(String namespace) throws PulsarAdminEx
     @Override
     public void setRetention(String namespace, RetentionPolicies retention) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("retention"))
                     .post(Entity.entity(retention, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -312,7 +312,7 @@ public void setRetention(String namespace, RetentionPolicies retention) throws P
     @Override
     public RetentionPolicies getRetention(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("retention"))
                             .get(RetentionPolicies.class);
@@ -324,7 +324,7 @@ public RetentionPolicies getRetention(String namespace) throws PulsarAdminExcept
     @Override
     public void unload(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("unload"))
                     .put(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -335,7 +335,7 @@ public void unload(String namespace) throws PulsarAdminException {
     @Override
     public String getReplicationConfigVersion(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
                     .path("configversion")).get(String.class);
         } catch (Exception e) {
@@ -346,7 +346,7 @@ public String getReplicationConfigVersion(String namespace) throws PulsarAdminEx
     @Override
     public void unloadNamespaceBundle(String namespace, String bundle) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle)
                     .path("unload")).put(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -358,7 +358,7 @@ public void unloadNamespaceBundle(String namespace, String bundle) throws Pulsar
     public void splitNamespaceBundle(String namespace, String bundle, boolean unloadSplitBundles)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle)
                     .path("split").queryParam("unload", Boolean.toString(unloadSplitBundles)))
                             .put(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
@@ -370,7 +370,7 @@ public void splitNamespaceBundle(String namespace, String bundle, boolean unload
     @Override
     public void setDispatchRate(String namespace, DispatchRate dispatchRate) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("dispatchRate"))
                     .post(Entity.entity(dispatchRate, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -381,7 +381,7 @@ public void setDispatchRate(String namespace, DispatchRate dispatchRate) throws
     @Override
     public DispatchRate getDispatchRate(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("dispatchRate"))
             .get(DispatchRate.class);
         } catch (Exception e) {
@@ -392,7 +392,7 @@ public DispatchRate getDispatchRate(String namespace) throws PulsarAdminExceptio
     @Override
     public void clearNamespaceBacklog(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
                     .path("clearBacklog")).post(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -404,7 +404,7 @@ public void clearNamespaceBacklog(String namespace) throws PulsarAdminException
     public void clearNamespaceBacklogForSubscription(String namespace, String subscription)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("clearBacklog")
                     .path(subscription)).post(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -415,7 +415,7 @@ public void clearNamespaceBacklogForSubscription(String namespace, String subscr
     @Override
     public void clearNamespaceBundleBacklog(String namespace, String bundle) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle)
                     .path("clearBacklog")).post(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -427,7 +427,7 @@ public void clearNamespaceBundleBacklog(String namespace, String bundle) throws
     public void clearNamespaceBundleBacklogForSubscription(String namespace, String bundle, String subscription)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle)
                     .path("clearBacklog").path(subscription)).post(Entity.entity("", MediaType.APPLICATION_JSON),
                             ErrorData.class);
@@ -439,7 +439,7 @@ public void clearNamespaceBundleBacklogForSubscription(String namespace, String
     @Override
     public void unsubscribeNamespace(String namespace, String subscription) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("unsubscribe")
                     .path(subscription)).post(Entity.entity("", MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -451,7 +451,7 @@ public void unsubscribeNamespace(String namespace, String subscription) throws P
     public void unsubscribeNamespaceBundle(String namespace, String bundle, String subscription)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle)
                     .path("unsubscribe").path(subscription)).post(Entity.entity("", MediaType.APPLICATION_JSON),
                             ErrorData.class);
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
index 48f57fe1e..cc1d96c41 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
@@ -85,7 +85,7 @@ public PersistentTopicsImpl(WebTarget web, Authentication auth) {
     @Override
     public List<String> getList(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(persistentTopics.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())).get(
                     new GenericType<List<String>>() {
                     });
@@ -97,7 +97,7 @@ public PersistentTopicsImpl(WebTarget web, Authentication auth) {
     @Override
     public List<String> getPartitionedTopicList(String namespace) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(persistentTopics.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path("partitioned")).get(
                     new GenericType<List<String>>() {
                     });
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java
index e6b41713c..f75afb336 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java
@@ -56,7 +56,7 @@ public void setDefaultResourceQuota(ResourceQuota quota) throws PulsarAdminExcep
 
     public ResourceQuota getNamespaceBundleResourceQuota(String namespace, String bundle) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             return request(
                     quotas.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle))
                     .get(ResourceQuota.class);
@@ -68,7 +68,7 @@ public ResourceQuota getNamespaceBundleResourceQuota(String namespace, String bu
     public void setNamespaceBundleResourceQuota(String namespace, String bundle, ResourceQuota quota)
             throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(
                 quotas.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle))
                     .post(Entity.entity(quota, MediaType.APPLICATION_JSON), ErrorData.class);
@@ -79,7 +79,7 @@ public void setNamespaceBundleResourceQuota(String namespace, String bundle, Res
 
     public void resetNamespaceBundleResourceQuota(String namespace, String bundle) throws PulsarAdminException {
         try {
-            NamespaceName ns = new NamespaceName(namespace);
+            NamespaceName ns = NamespaceName.get(namespace);
             request(
                 quotas.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName()).path(bundle))
                     .delete();
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java
index e6bfd8ee1..c3e018f6a 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java
@@ -40,7 +40,7 @@
 
     String validateNamespace(List<String> params) {
         String namespace = checkArgument(params);
-        return new NamespaceName(namespace).toString();
+        return NamespaceName.get(namespace).toString();
     }
 
     String validateDestination(List<String> params) {
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 69083c584..350501071 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
@@ -22,6 +22,7 @@
 
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.pulsar.common.util.Codec;
 import org.slf4j.Logger;
@@ -56,7 +57,7 @@
     private final int partitionIndex;
 
     private static final LoadingCache<String, DestinationName> cache = CacheBuilder.newBuilder().maximumSize(100000)
-            .build(new CacheLoader<String, DestinationName>() {
+            .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader<String, DestinationName>() {
                 @Override
                 public DestinationName load(String name) throws Exception {
                     return new DestinationName(name);
@@ -123,7 +124,7 @@ private DestinationName(String destination) {
         } catch (NullPointerException e) {
             throw new IllegalArgumentException("Invalid destination name: " + destination, e);
         }
-        namespaceName = new NamespaceName(property, cluster, namespacePortion);
+        namespaceName = NamespaceName.get(property, cluster, namespacePortion);
     }
 
     /**
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 f9dfdd94c..a179f9af5 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
@@ -22,8 +22,14 @@
 
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Objects;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.UncheckedExecutionException;
 
 public class NamespaceName implements ServiceUnitId {
 
@@ -32,8 +38,36 @@
     private String property;
     private String cluster;
     private String localName;
+    
+    private static final LoadingCache<String, NamespaceName> cache = CacheBuilder.newBuilder().maximumSize(100000)
+            .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader<String, NamespaceName>() {
+                @Override
+                public NamespaceName load(String name) throws Exception {
+                    return new NamespaceName(name);
+                }
+            });
+
+    public static NamespaceName get(String property, String cluster, String namespace) {
+        validateNamespaceName(property, cluster, namespace);
+        return get(property + '/' + cluster + '/' + namespace);
+    }
 
-    public NamespaceName(String namespace) {
+    public static NamespaceName get(String namespace) {
+        try {
+            checkNotNull(namespace);
+        } catch (NullPointerException e) {
+            throw new IllegalArgumentException("Invalid null namespace: " + namespace);
+        }
+        try {
+            return cache.get(namespace);
+        } catch (ExecutionException e) {
+            throw (RuntimeException) e.getCause();
+        } catch (UncheckedExecutionException e) {
+            throw (RuntimeException) e.getCause();
+        }
+    }
+
+    private NamespaceName(String namespace) {
         try {
             checkNotNull(namespace);
         } catch (NullPointerException e) {
@@ -45,14 +79,6 @@ public NamespaceName(String namespace) {
         this.namespace = namespace;
     }
 
-    public NamespaceName(String property, String cluster, String namespace) {
-        validateNamespaceName(property, cluster, namespace);
-        this.namespace = property + '/' + cluster + '/' + namespace;
-        this.property = property;
-        this.cluster = cluster;
-        this.localName = namespace;
-    }
-
     public String getProperty() {
         return property;
     }
@@ -162,4 +188,5 @@ public NamespaceName getNamespaceObject() {
     public boolean includes(DestinationName dn) {
         return this.equals(dn.getNamespaceObject());
     }
+
 }
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 d52e7da85..7d6a7cf0b 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
@@ -32,172 +32,172 @@
     @Test
     void namespace() {
         try {
-            new NamespaceName("namespace");
+            NamespaceName.get("namespace");
             fail("Should have caused exception");
         } catch (IllegalArgumentException e) {
             // expected
         }
 
         try {
-            new NamespaceName("property.namespace");
+            NamespaceName.get("property.namespace");
             fail("Should have caused exception");
         } catch (IllegalArgumentException e) {
             // expected
         }
 
         try {
-            new NamespaceName("0.0.0.0");
+            NamespaceName.get("0.0.0.0");
             fail("Should have caused exception");
         } catch (IllegalArgumentException e) {
             // expected
         }
 
         try {
-            new NamespaceName("property.namespace:destination");
+            NamespaceName.get("property.namespace:destination");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("property/namespace");
+            NamespaceName.get("property/namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("property/cluster/namespace/destination");
+            NamespaceName.get("property/cluster/namespace/destination");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName(null);
+            NamespaceName.get(null);
         } catch (IllegalArgumentException e) {
             // OK
         }
 
         try {
-            new NamespaceName(null, "use", "ns1");
+            NamespaceName.get(null, "use", "ns1");
         } catch (IllegalArgumentException e) {
             // OK
         }
 
-        assertEquals(new NamespaceName("prop/cluster/ns").getPersistentTopicName("ds"),
+        assertEquals(NamespaceName.get("prop/cluster/ns").getPersistentTopicName("ds"),
                 "persistent://prop/cluster/ns/ds");
 
         try {
-            new NamespaceName("prop/cluster/ns").getDestinationName(null, "ds");
+            NamespaceName.get("prop/cluster/ns").getDestinationName(null, "ds");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
-        assertEquals(new NamespaceName("prop/cluster/ns").getDestinationName(DestinationDomain.persistent, "ds"),
+        assertEquals(NamespaceName.get("prop/cluster/ns").getDestinationName(DestinationDomain.persistent, "ds"),
                 "persistent://prop/cluster/ns/ds");
-        assertEquals(new NamespaceName("prop/cluster/ns"), new NamespaceName("prop/cluster/ns"));
-        assertEquals(new NamespaceName("prop/cluster/ns").toString(), "prop/cluster/ns");
-        assertFalse(new NamespaceName("prop/cluster/ns").equals("prop/cluster/ns"));
+        assertEquals(NamespaceName.get("prop/cluster/ns"), NamespaceName.get("prop/cluster/ns"));
+        assertEquals(NamespaceName.get("prop/cluster/ns").toString(), "prop/cluster/ns");
+        assertFalse(NamespaceName.get("prop/cluster/ns").equals("prop/cluster/ns"));
 
-        assertEquals(new NamespaceName("prop", "cluster", "ns"), new NamespaceName("prop/cluster/ns"));
-        assertEquals(new NamespaceName("prop/cluster/ns").getProperty(), "prop");
-        assertEquals(new NamespaceName("prop/cluster/ns").getCluster(), "cluster");
-        assertEquals(new NamespaceName("prop/cluster/ns").getLocalName(), "ns");
+        assertEquals(NamespaceName.get("prop", "cluster", "ns"), NamespaceName.get("prop/cluster/ns"));
+        assertEquals(NamespaceName.get("prop/cluster/ns").getProperty(), "prop");
+        assertEquals(NamespaceName.get("prop/cluster/ns").getCluster(), "cluster");
+        assertEquals(NamespaceName.get("prop/cluster/ns").getLocalName(), "ns");
 
         try {
-            new NamespaceName("ns").getProperty();
+            NamespaceName.get("ns").getProperty();
             fail("old style namespace");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("ns").getCluster();
+            NamespaceName.get("ns").getCluster();
             fail("old style namespace");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("ns").getLocalName();
+            NamespaceName.get("ns").getLocalName();
             fail("old style namespace");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("_pulsar/cluster/namespace");
+            NamespaceName.get("_pulsar/cluster/namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName(null, "cluster", "namespace");
+            NamespaceName.get(null, "cluster", "namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("", "cluster", "namespace");
+            NamespaceName.get("", "cluster", "namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("/cluster/namespace");
+            NamespaceName.get("/cluster/namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar//namespace");
+            NamespaceName.get("pulsar//namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar", null, "namespace");
+            NamespaceName.get("pulsar", null, "namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar", "", "namespace");
+            NamespaceName.get("pulsar", "", "namespace");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar/cluster/");
+            NamespaceName.get("pulsar/cluster/");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar", "cluster", null);
+            NamespaceName.get("pulsar", "cluster", null);
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
         try {
-            new NamespaceName("pulsar", "cluster", "");
+            NamespaceName.get("pulsar", "cluster", "");
             fail("Should have raised exception");
         } catch (IllegalArgumentException e) {
             // Ok
         }
 
-        NamespaceName v2Namespace = new NamespaceName("pulsar/colo1/testns-1");
+        NamespaceName v2Namespace = NamespaceName.get("pulsar/colo1/testns-1");
         assertEquals(v2Namespace.getProperty(), "pulsar");
         assertEquals(v2Namespace.getCluster(), "colo1");
         assertEquals(v2Namespace.getLocalName(), "testns-1");
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java
index 23aafce2f..a8cbd7bff 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPoliciesTest.java
@@ -120,9 +120,9 @@ public void testGetNamespaceIsolationPolicyByName() throws Exception {
     @Test
     public void testGetNamespaceIsolationPolicyByNamespace() throws Exception {
         NamespaceIsolationPolicies policies = this.getDefaultTestPolicies();
-        NamespaceIsolationPolicy nsPolicy = policies.getPolicyByNamespace(new NamespaceName("no/such/namespace"));
+        NamespaceIsolationPolicy nsPolicy = policies.getPolicyByNamespace(NamespaceName.get("no/such/namespace"));
         assertTrue(nsPolicy == null);
-        nsPolicy = policies.getPolicyByNamespace(new NamespaceName("pulsar/use/testns-1"));
+        nsPolicy = policies.getPolicyByNamespace(NamespaceName.get("pulsar/use/testns-1"));
         assertNotNull(nsPolicy);
         assertEquals(new NamespaceIsolationPolicyImpl(policies.getPolicies().get("policy1")), nsPolicy);
     }
@@ -141,7 +141,7 @@ public void testSetPolicy() throws Exception {
         assertEquals(policies.getPolicies().size(), 2);
         assertEquals(policies.getPolicyByName(newPolicyName), new NamespaceIsolationPolicyImpl(nsPolicyData));
         assertTrue(!policies.getPolicyByName(newPolicyName).equals(policies.getPolicyByName("policy1")));
-        assertEquals(policies.getPolicyByNamespace(new NamespaceName("pulsar/use/TESTNS.1")),
+        assertEquals(policies.getPolicyByNamespace(NamespaceName.get("pulsar/use/TESTNS.1")),
                 new NamespaceIsolationPolicyImpl(nsPolicyData));
     }
 
@@ -156,7 +156,7 @@ private NamespaceIsolationPolicies getDefaultTestPolicies() throws Exception {
     @Test
     public void testBrokerAssignment() throws Exception {
         NamespaceIsolationPolicies policies = this.getDefaultTestPolicies();
-        NamespaceName ns = new NamespaceName("pulsar/use/testns-1");
+        NamespaceName ns = NamespaceName.get("pulsar/use/testns-1");
         SortedSet<BrokerStatus> primaryCandidates = new TreeSet<>();
         BrokerStatus primary = new BrokerStatus("prod1-broker1.messaging.use.example.com", true, 0);
         BrokerStatus secondary = new BrokerStatus("prod1-broker4.use.example.com", true, 0);
@@ -173,7 +173,7 @@ public void testBrokerAssignment() throws Exception {
         assertEquals(secondaryCandidates.size(), 1);
         assertEquals(sharedCandidates.size(), 0);
         assertTrue(secondaryCandidates.first().equals(secondary));
-        policies.assignBroker(new NamespaceName("pulsar/use1/testns-1"), shared, primaryCandidates, secondaryCandidates,
+        policies.assignBroker(NamespaceName.get("pulsar/use1/testns-1"), shared, primaryCandidates, secondaryCandidates,
                 sharedCandidates);
         assertEquals(primaryCandidates.size(), 1);
         assertEquals(secondaryCandidates.size(), 1);
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java
index 728343a21..1aee82f01 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImplTest.java
@@ -110,28 +110,28 @@ public void testFindBrokers() throws Exception {
             String broker = String.format("prod1-broker%d.messaging.usw.example.com", i);
             brokers.add(new URL(String.format("http://%s:8080", broker)));
         }
-        List<URL> primaryBrokers = defaultPolicy.findPrimaryBrokers(brokers, new NamespaceName("pulsar/use/testns-1"));
+        List<URL> primaryBrokers = defaultPolicy.findPrimaryBrokers(brokers, NamespaceName.get("pulsar/use/testns-1"));
         assertEquals(primaryBrokers.size(), 3);
         for (URL primaryBroker : primaryBrokers) {
             assertTrue(primaryBroker.getHost().matches("prod1-broker[1-3].messaging.use.example.com"));
         }
-        primaryBrokers = defaultPolicy.findPrimaryBrokers(otherBrokers, new NamespaceName("pulsar/use/testns-1"));
+        primaryBrokers = defaultPolicy.findPrimaryBrokers(otherBrokers, NamespaceName.get("pulsar/use/testns-1"));
         assertTrue(primaryBrokers.isEmpty());
         try {
-            primaryBrokers = defaultPolicy.findPrimaryBrokers(brokers, new NamespaceName("no/such/namespace"));
+            primaryBrokers = defaultPolicy.findPrimaryBrokers(brokers, NamespaceName.get("no/such/namespace"));
         } catch (IllegalArgumentException iae) {
             // OK
         }
         List<URL> secondaryBrokers = defaultPolicy.findSecondaryBrokers(brokers,
-                new NamespaceName("pulsar/use/testns-1"));
+                NamespaceName.get("pulsar/use/testns-1"));
         assertEquals(secondaryBrokers.size(), 10);
         for (URL secondaryBroker : secondaryBrokers) {
             assertTrue(secondaryBroker.getHost().matches("prod1-broker.*.messaging.use.example.com"));
         }
-        secondaryBrokers = defaultPolicy.findSecondaryBrokers(otherBrokers, new NamespaceName("pulsar/use/testns-1"));
+        secondaryBrokers = defaultPolicy.findSecondaryBrokers(otherBrokers, NamespaceName.get("pulsar/use/testns-1"));
         assertTrue(secondaryBrokers.isEmpty());
         try {
-            secondaryBrokers = defaultPolicy.findSecondaryBrokers(brokers, new NamespaceName("no/such/namespace"));
+            secondaryBrokers = defaultPolicy.findSecondaryBrokers(brokers, NamespaceName.get("no/such/namespace"));
         } catch (IllegalArgumentException iae) {
             // OK
         }


 

----------------------------------------------------------------
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

Mime
View raw message