You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by mm...@apache.org on 2018/02/26 04:57:43 UTC

[incubator-pulsar] branch master updated: Renamed DestinationName into TopicName (#1280)

This is an automated email from the ASF dual-hosted git repository.

mmerli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new 5fc4d53  Renamed DestinationName into TopicName (#1280)
5fc4d53 is described below

commit 5fc4d536d76b1d73c6be7b9238bf14b210ee095f
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Sun Feb 25 20:57:40 2018 -0800

    Renamed DestinationName into TopicName (#1280)
---
 .../mledger/impl/ManagedLedgerOfflineBacklog.java  |  20 +-
 .../authorization/AuthorizationProvider.java       |  39 +-
 .../broker/authorization/AuthorizationService.java | 154 ++++----
 .../authorization/PulsarAuthorizationProvider.java | 102 ++---
 .../org/apache/pulsar/broker/PulsarService.java    |  12 +-
 .../apache/pulsar/broker/admin/AdminResource.java  |  32 +-
 .../pulsar/broker/admin/impl/BrokerStatsBase.java  |   8 +-
 .../pulsar/broker/admin/impl/NamespacesBase.java   |  16 +-
 .../broker/admin/impl/PersistentTopicsBase.java    | 440 ++++++++++-----------
 .../pulsar/broker/admin/impl/PropertiesBase.java   |   2 +-
 .../apache/pulsar/broker/admin/v1/Namespaces.java  |  31 +-
 .../broker/admin/v1/NonPersistentTopics.java       |  71 ++--
 .../pulsar/broker/admin/v1/PersistentTopics.java   | 172 ++++----
 .../apache/pulsar/broker/admin/v2/Namespaces.java  |  22 +-
 .../broker/admin/v2/NonPersistentTopics.java       |  68 ++--
 .../pulsar/broker/admin/v2/PersistentTopics.java   | 166 ++++----
 .../loadbalance/impl/SimpleLoadCalculatorImpl.java |   2 +-
 .../{DestinationLookup.java => TopicLookup.java}   |  78 ++--
 .../pulsar/broker/namespace/NamespaceService.java  |  54 +--
 .../pulsar/broker/service/AbstractReplicator.java  |  22 +-
 .../pulsar/broker/service/BacklogQuotaManager.java |   6 +-
 .../pulsar/broker/service/BrokerService.java       |  65 ++-
 .../org/apache/pulsar/broker/service/Consumer.java |  11 +-
 .../org/apache/pulsar/broker/service/Producer.java |  10 +-
 .../apache/pulsar/broker/service/PulsarStats.java  |  10 +-
 .../apache/pulsar/broker/service/ServerCnx.java    |  20 +-
 .../apache/pulsar/broker/service/Subscription.java |   2 +-
 .../nonpersistent/NonPersistentSubscription.java   |   6 +-
 .../service/nonpersistent/NonPersistentTopic.java  |  86 ++--
 .../service/persistent/CompactorSubscription.java  |   2 +-
 .../service/persistent/DispatchRateLimiter.java    |   6 +-
 .../service/persistent/MessageDeduplication.java   |   4 +-
 .../service/persistent/PersistentSubscription.java |   6 +-
 .../broker/service/persistent/PersistentTopic.java | 136 +++----
 .../broker/stats/BookieClientStatsGenerator.java   |  16 +-
 .../pulsar/broker/stats/MetricsGenerator.java      |   2 +-
 .../broker/stats/metrics/AbstractMetrics.java      |  14 +-
 .../broker/stats/metrics/ManagedLedgerMetrics.java |  16 +-
 .../pulsar/broker/web/PulsarWebResource.java       |  34 +-
 .../pulsar/common/naming/NamespaceBundle.java      |   8 +-
 .../pulsar/common/naming/NamespaceBundles.java     |  10 +-
 .../apache/pulsar/broker/SLAMonitoringTest.java    |  20 +-
 .../apache/pulsar/broker/admin/AdminApiTest.java   |  58 +--
 .../apache/pulsar/broker/admin/AdminApiTest2.java  |  16 +-
 .../org/apache/pulsar/broker/admin/AdminTest.java  |  22 +-
 .../broker/admin/CreateSubscriptionTest.java       |   4 +-
 .../broker/admin/IncrementPartitionsTest.java      |   4 +-
 .../apache/pulsar/broker/admin/NamespacesTest.java |  20 +-
 .../pulsar/broker/auth/AuthorizationTest.java      | 158 ++++----
 .../broker/loadbalance/LoadBalancerTest.java       |  24 +-
 ...ookupv2Test.java => HttpTopicLookupv2Test.java} |  20 +-
 .../broker/namespace/NamespaceServiceTest.java     |  30 +-
 .../broker/service/BrokerBkEnsemblesTests.java     |  30 +-
 .../pulsar/broker/service/BrokerServiceTest.java   |  22 +-
 .../PersistentDispatcherFailoverConsumerTest.java  |   4 +-
 .../broker/service/PersistentFailoverE2ETest.java  |   4 +-
 .../service/PersistentTopicConcurrentTest.java     |   4 +-
 .../broker/service/PersistentTopicE2ETest.java     |  18 +-
 .../pulsar/broker/service/PersistentTopicTest.java |   6 +-
 .../pulsar/broker/service/ReplicatorTest.java      |  36 +-
 .../pulsar/broker/service/ReplicatorTestBase.java  |  18 +-
 .../pulsar/broker/service/ServerCnxTest.java       |  24 +-
 .../zookeeper/ZooKeeperClientAspectJTest.java      |   4 +-
 .../api/AuthenticatedProducerConsumerTest.java     |   6 +-
 .../api/AuthorizationProducerConsumerTest.java     |  50 +--
 .../pulsar/client/api/BrokerServiceLookupTest.java |  30 +-
 .../pulsar/client/api/NonPersistentTopicTest.java  |  18 +-
 .../api/PartitionedProducerConsumerTest.java       |  97 +++--
 .../client/api/SimpleProducerConsumerTest.java     |   8 +-
 .../client/impl/BrokerClientIntegrationTest.java   |  28 +-
 .../pulsar/common/naming/NamespaceBundleTest.java  |  14 +-
 .../pulsar/common/naming/NamespaceBundlesTest.java |  26 +-
 .../websocket/proxy/ProxyAuthorizationTest.java    |  26 +-
 .../apache/pulsar/client/admin/BrokerStats.java    |  12 +-
 .../org/apache/pulsar/client/admin/Lookup.java     |  21 +-
 .../org/apache/pulsar/client/admin/Namespaces.java |  34 +-
 .../pulsar/client/admin/NonPersistentTopics.java   |  72 ++--
 .../pulsar/client/admin/PersistentTopics.java      | 294 +++++++-------
 .../org/apache/pulsar/client/admin/Properties.java |   2 +-
 .../apache/pulsar/client/admin/PulsarAdmin.java    |   6 +-
 .../client/admin/internal/BrokerStatsImpl.java     |   2 +-
 .../pulsar/client/admin/internal/LookupImpl.java   |  14 +-
 .../client/admin/internal/NamespacesImpl.java      |   2 +-
 .../admin/internal/NonPersistentTopicsImpl.java    |  48 +--
 .../admin/internal/PersistentTopicsImpl.java       | 194 ++++-----
 .../clients/consumer/PulsarKafkaConsumer.java      |  10 +-
 .../org/apache/pulsar/admin/cli/CliCommand.java    |  26 +-
 .../apache/pulsar/admin/cli/CmdBrokerStats.java    |  10 +-
 .../org/apache/pulsar/admin/cli/CmdNamespaces.java |  11 +-
 .../pulsar/admin/cli/CmdNonPersistentTopics.java   |  20 +-
 .../pulsar/admin/cli/CmdPersistentTopics.java      |  84 ++--
 .../pulsar/admin/cli/PulsarAdminToolTest.java      |  10 +-
 .../client/impl/BinaryProtoLookupService.java      |  30 +-
 .../pulsar/client/impl/HttpLookupService.java      |  14 +-
 .../apache/pulsar/client/impl/LookupService.java   |  10 +-
 .../client/impl/PartitionedConsumerImpl.java       |   4 +-
 .../client/impl/PartitionedProducerImpl.java       |   4 +-
 .../pulsar/client/impl/PulsarClientImpl.java       |  20 +-
 .../pulsar/client/impl/TopicsConsumerImpl.java     |  28 +-
 .../apache/pulsar/common/naming/NamedEntity.java   |   2 +-
 .../apache/pulsar/common/naming/NamespaceName.java |  18 +-
 .../apache/pulsar/common/naming/ServiceUnitId.java |   8 +-
 .../{DestinationDomain.java => TopicDomain.java}   |   8 +-
 .../{DestinationName.java => TopicName.java}       | 104 ++---
 .../pulsar/common/naming/NamespaceNameTest.java    |  10 +-
 ...DestinationNameTest.java => TopicNameTest.java} | 130 +++---
 .../discovery/service/BrokerDiscoveryProvider.java |  22 +-
 .../pulsar/discovery/service/ServerConnection.java |  10 +-
 .../discovery/service/DiscoveryServiceTest.java    |   6 +-
 .../proxy/server/BrokerDiscoveryProvider.java      |  22 +-
 .../pulsar/proxy/server/LookupProxyHandler.java    |  18 +-
 .../proxy/socket/client/PerformanceClient.java     |  10 +-
 .../pulsar/testclient/LoadSimulationClient.java    |   8 +-
 .../testclient/LoadSimulationController.java       |  70 ++--
 .../pulsar/testclient/PerformanceConsumer.java     |  24 +-
 .../pulsar/testclient/PerformanceProducer.java     |   8 +-
 .../pulsar/testclient/PerformanceReader.java       |  16 +-
 .../pulsar/websocket/AbstractWebSocketHandler.java |   6 +-
 .../apache/pulsar/websocket/ConsumerHandler.java   |   4 +-
 .../apache/pulsar/websocket/ProducerHandler.java   |   4 +-
 .../org/apache/pulsar/websocket/ReaderHandler.java |   4 +-
 .../websocket/admin/WebSocketProxyStats.java       |  18 +-
 .../websocket/admin/WebSocketWebResource.java      |   6 +-
 .../apache/pulsar/websocket/stats/ProxyStats.java  |   6 +-
 124 files changed, 2172 insertions(+), 2182 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
index ee22c5e..c07ed55 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
@@ -36,7 +36,7 @@ import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.proto.MLDataFormats;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
 import org.slf4j.Logger;
@@ -100,12 +100,12 @@ public class ManagedLedgerOfflineBacklog {
 
     public PersistentOfflineTopicStats getEstimatedUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory,
             String managedLedgerName) throws Exception {
-        return estimateUnloadedTopicBacklog(factory, DestinationName.get("persistent://" + managedLedgerName));
+        return estimateUnloadedTopicBacklog(factory, TopicName.get("persistent://" + managedLedgerName));
     }
 
     public PersistentOfflineTopicStats estimateUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory,
-            DestinationName dn) throws Exception {
-        String managedLedgerName = dn.getPersistenceNamingEncoding();
+            TopicName topicName) throws Exception {
+        String managedLedgerName = topicName.getPersistenceNamingEncoding();
         long numberOfEntries = 0;
         long totalSize = 0;
         final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers = new ConcurrentSkipListMap<>();
@@ -113,7 +113,7 @@ public class ManagedLedgerOfflineBacklog {
                 brokerName);
 
         // calculate total managed ledger size and number of entries without loading the topic
-        readLedgerMeta(factory, dn, ledgers);
+        readLedgerMeta(factory, topicName, ledgers);
         for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers.values()) {
             numberOfEntries += ls.getEntries();
             totalSize += ls.getSize();
@@ -128,15 +128,15 @@ public class ManagedLedgerOfflineBacklog {
         }
 
         // calculate per cursor message backlog
-        calculateCursorBacklogs(factory, dn, ledgers, offlineTopicStats);
+        calculateCursorBacklogs(factory, topicName, ledgers, offlineTopicStats);
         offlineTopicStats.statGeneratedAt.setTime(System.currentTimeMillis());
 
         return offlineTopicStats;
     }
 
-    private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final DestinationName dn,
+    private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final TopicName topicName,
             final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers) throws Exception {
-        String managedLedgerName = dn.getPersistenceNamingEncoding();
+        String managedLedgerName = topicName.getPersistenceNamingEncoding();
         MetaStore store = factory.getMetaStore();
         BookKeeper bk = factory.getBookKeeper();
         final CountDownLatch mlMetaCounter = new CountDownLatch(1);
@@ -206,14 +206,14 @@ public class ManagedLedgerOfflineBacklog {
         }
     }
 
-    private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, final DestinationName dn,
+    private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, final TopicName topicName,
             final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers,
             final PersistentOfflineTopicStats offlineTopicStats) throws Exception {
 
         if (ledgers.size() == 0) {
             return;
         }
-        String managedLedgerName = dn.getPersistenceNamingEncoding();
+        String managedLedgerName = topicName.getPersistenceNamingEncoding();
         MetaStore store = factory.getMetaStore();
         BookKeeper bk = factory.getBookKeeper();
         final CountDownLatch allCursorsCounter = new CountDownLatch(1);
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
index 9962c05..cae2415 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
@@ -26,7 +26,7 @@ import java.util.concurrent.CompletableFuture;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 
@@ -48,47 +48,46 @@ public interface AuthorizationProvider extends Closeable {
     void initialize(ServiceConfiguration conf, ConfigurationCacheService configCache) throws IOException;
 
     /**
-     * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+     * Check if the specified role has permission to send messages to the specified fully qualified topic name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to send messages to the destination.
+     *            the app id used to send messages to the topic.
      */
-    CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+    CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData);
 
     /**
-     * Check if the specified role has permission to receive messages from the specified fully qualified destination
-     * name.
+     * Check if the specified role has permission to receive messages from the specified fully qualified topic name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to receive messages from the destination.
+     *            the app id used to receive messages from the topic.
      * @param subscription
      *            the subscription name defined by the client
      */
-    CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+    CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData, String subscription);
 
     /**
-     * Check whether the specified role can perform a lookup for the specified destination.
+     * Check whether the specified role can perform a lookup for the specified topic.
      *
      * For that the caller needs to have producer or consumer permission.
      *
-     * @param destination
+     * @param topicName
      * @param role
      * @return
      * @throws Exception
      */
-    CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+    CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData);
 
     /**
-     * 
+     *
      * Grant authorization-action permission on a namespace to the given client
-     * 
+     *
      * @param namespace
      * @param actions
      * @param role
@@ -104,8 +103,8 @@ public interface AuthorizationProvider extends Closeable {
 
     /**
      * Grant authorization-action permission on a topic to the given client
-     * 
-     * @param topicname
+     *
+     * @param topicName
      * @param role
      * @param authDataJson
      *            additional authdata in json format
@@ -114,7 +113,7 @@ public interface AuthorizationProvider extends Closeable {
      *                IllegalArgumentException when namespace not found<br/>
      *                IllegalStateException when failed to grant permission
      */
-    CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions, String role,
+    CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<AuthAction> actions, String role,
             String authDataJson);
 
 }
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
index 8cb9232..f38482d 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
@@ -34,7 +34,7 @@ import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.util.FutureUtil;
@@ -45,7 +45,7 @@ import com.google.common.collect.Maps;
 
 /**
  * Authorization service that manages pluggable authorization provider and authorize requests accordingly.
- * 
+ *
  */
 public class AuthorizationService {
     private static final Logger log = LoggerFactory.getLogger(AuthorizationService.class);
@@ -60,16 +60,16 @@ public class AuthorizationService {
         if (this.conf.isAuthorizationEnabled()) {
             try {
                 final String providerClassname = conf.getAuthorizationProvider();
-                if(StringUtils.isNotBlank(providerClassname)) {
+                if (StringUtils.isNotBlank(providerClassname)) {
                     provider = (AuthorizationProvider) Class.forName(providerClassname).newInstance();
                     provider.initialize(conf, configCache);
-                    log.info("{} has been loaded.", providerClassname); 
+                    log.info("{} has been loaded.", providerClassname);
                 } else {
                     throw new PulsarServerException("No authorization providers are present.");
                 }
             } catch (PulsarServerException e) {
                 throw e;
-            }catch (Throwable e) {
+            } catch (Throwable e) {
                 throw new PulsarServerException("Failed to load an authorization provider.", e);
             }
         } else {
@@ -78,9 +78,9 @@ public class AuthorizationService {
     }
 
     /**
-     * 
+     *
      * Grant authorization-action permission on a namespace to the given client
-     * 
+     *
      * @param namespace
      * @param actions
      * @param role
@@ -103,7 +103,7 @@ public class AuthorizationService {
 
     /**
      * Grant authorization-action permission on a topic to the given client
-     * 
+     *
      * @param topicname
      * @param role
      * @param authDataJson
@@ -112,26 +112,25 @@ public class AuthorizationService {
      * @throws IllegalStateException
      *             when failed to grant permission
      */
-    public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions, String role,
+    public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions, String role,
             String authDataJson) {
 
         if (provider != null) {
             return provider.grantPermissionAsync(topicname, actions, role, authDataJson);
         }
-        return FutureUtil
-                .failedFuture(new IllegalStateException("No authorization provider configured"));
+        return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
 
     }
 
     /**
-     * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+     * Check if the specified role has permission to send messages to the specified fully qualified topic name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to send messages to the destination.
+     *            the app id used to send messages to the topic.
      */
-    public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData) {
 
         if (!this.conf.isAuthorizationEnabled()) {
@@ -139,126 +138,121 @@ public class AuthorizationService {
         }
 
         if (provider != null) {
-            return provider.canProduceAsync(destination, role, authenticationData);
+            return provider.canProduceAsync(topicName, role, authenticationData);
         }
-        return FutureUtil
-                .failedFuture(new IllegalStateException("No authorization provider configured"));
+        return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
     }
 
     /**
-     * Check if the specified role has permission to receive messages from the specified fully qualified destination
-     * name.
+     * Check if the specified role has permission to receive messages from the specified fully qualified topic name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to receive messages from the destination.
+     *            the app id used to receive messages from the topic.
      * @param subscription
      *            the subscription name defined by the client
      */
-    public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData, String subscription) {
         if (!this.conf.isAuthorizationEnabled()) {
             return CompletableFuture.completedFuture(true);
         }
         if (provider != null) {
-            return provider.canConsumeAsync(destination, role, authenticationData, subscription);
+            return provider.canConsumeAsync(topicName, role, authenticationData, subscription);
         }
-        return FutureUtil
-                .failedFuture(new IllegalStateException("No authorization provider configured"));
+        return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
     }
 
-    public boolean canProduce(DestinationName destination, String role, AuthenticationDataSource authenticationData) throws Exception {
+    public boolean canProduce(TopicName topicName, String role, AuthenticationDataSource authenticationData)
+            throws Exception {
         try {
-            return canProduceAsync(destination, role, authenticationData).get(cacheTimeOutInSec,
-                    SECONDS);
+            return canProduceAsync(topicName, role, authenticationData).get(cacheTimeOutInSec, SECONDS);
         } catch (InterruptedException e) {
-            log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, destination);
+            log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, topicName);
             throw e;
         } catch (Exception e) {
-            log.warn("Producer-client  with Role - {} failed to get permissions for destination - {}. {}", role,
-                    destination, e.getMessage());
+            log.warn("Producer-client  with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
+                    e.getMessage());
             throw e;
         }
     }
 
-    public boolean canConsume(DestinationName destination, String role, AuthenticationDataSource authenticationData,
+    public boolean canConsume(TopicName topicName, String role, AuthenticationDataSource authenticationData,
             String subscription) throws Exception {
         try {
-            return canConsumeAsync(destination, role, authenticationData, subscription)
-                    .get(cacheTimeOutInSec, SECONDS);
+            return canConsumeAsync(topicName, role, authenticationData, subscription).get(cacheTimeOutInSec, SECONDS);
         } catch (InterruptedException e) {
-            log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, destination);
+            log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, topicName);
             throw e;
         } catch (Exception e) {
-            log.warn("Consumer-client  with Role - {} failed to get permissions for destination - {}. {}", role,
-                    destination, e.getMessage());
+            log.warn("Consumer-client  with Role - {} failed to get permissions for topic - {}. {}", role,
+                    topicName, e.getMessage());
             throw e;
         }
     }
 
     /**
-     * Check whether the specified role can perform a lookup for the specified destination.
+     * Check whether the specified role can perform a lookup for the specified topic.
      *
      * For that the caller needs to have producer or consumer permission.
      *
-     * @param destination
+     * @param topicName
      * @param role
      * @return
      * @throws Exception
      */
-    public boolean canLookup(DestinationName destination, String role, AuthenticationDataSource authenticationData) throws Exception {
-        return canProduce(destination, role, authenticationData)
-                || canConsume(destination, role, authenticationData, null);
+    public boolean canLookup(TopicName topicName, String role, AuthenticationDataSource authenticationData)
+            throws Exception {
+        return canProduce(topicName, role, authenticationData)
+                || canConsume(topicName, role, authenticationData, null);
     }
 
     /**
-     * Check whether the specified role can perform a lookup for the specified destination.
+     * Check whether the specified role can perform a lookup for the specified topic.
      *
      * For that the caller needs to have producer or consumer permission.
      *
-     * @param destination
+     * @param topicName
      * @param role
      * @return
      * @throws Exception
      */
-    public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData) {
         CompletableFuture<Boolean> finalResult = new CompletableFuture<Boolean>();
-        canProduceAsync(destination, role, authenticationData)
-                .whenComplete((produceAuthorized, ex) -> {
-                    if (ex == null) {
-                        if (produceAuthorized) {
-                            finalResult.complete(produceAuthorized);
-                            return;
-                        }
-                    } else {
-                        if (log.isDebugEnabled()) {
-                            log.debug(
-                                    "Destination [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
-                                    destination.toString(), role, ex.getMessage());
-                        }
+        canProduceAsync(topicName, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
+            if (ex == null) {
+                if (produceAuthorized) {
+                    finalResult.complete(produceAuthorized);
+                    return;
+                }
+            } else {
+                if (log.isDebugEnabled()) {
+                    log.debug(
+                            "Topic [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
+                            topicName.toString(), role, ex.getMessage());
+                }
+            }
+            canConsumeAsync(topicName, role, null, null).whenComplete((consumeAuthorized, e) -> {
+                if (e == null) {
+                    if (consumeAuthorized) {
+                        finalResult.complete(consumeAuthorized);
+                        return;
                     }
-                    canConsumeAsync(destination, role, null, null)
-                            .whenComplete((consumeAuthorized, e) -> {
-                                if (e == null) {
-                                    if (consumeAuthorized) {
-                                        finalResult.complete(consumeAuthorized);
-                                        return;
-                                    }
-                                } else {
-                                    if (log.isDebugEnabled()) {
-                                        log.debug(
-                                                "Destination [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
-                                                destination.toString(), role, e.getMessage());
+                } else {
+                    if (log.isDebugEnabled()) {
+                        log.debug(
+                                "Topic [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
+                                topicName.toString(), role, e.getMessage());
 
-                                    }
-                                    finalResult.completeExceptionally(e);
-                                    return;
-                                }
-                                finalResult.complete(false);
-                            });
-                });
+                    }
+                    finalResult.completeExceptionally(e);
+                    return;
+                }
+                finalResult.complete(false);
+            });
+        });
         return finalResult;
     }
 
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
index 7bd9246..a52d5c9 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
@@ -31,7 +31,7 @@ import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.Policies;
@@ -74,39 +74,39 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
     }
 
     /**
-     * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+     * Check if the specified role has permission to send messages to the specified fully qualified topic name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to send messages to the destination.
+     *            the app id used to send messages to the topic.
      */
     @Override
-    public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData) {
-        return checkAuthorization(destination, role, AuthAction.produce);
+        return checkAuthorization(topicName, role, AuthAction.produce);
     }
 
     /**
-     * Check if the specified role has permission to receive messages from the specified fully qualified destination
+     * Check if the specified role has permission to receive messages from the specified fully qualified topic
      * name.
      *
-     * @param destination
-     *            the fully qualified destination name associated with the destination.
+     * @param topicName
+     *            the fully qualified topic name associated with the topic.
      * @param role
-     *            the app id used to receive messages from the destination.
+     *            the app id used to receive messages from the topic.
      * @param subscription
      *            the subscription name defined by the client
      */
     @Override
-    public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData, String subscription) {
         CompletableFuture<Boolean> permissionFuture = new CompletableFuture<>();
         try {
-            configCache.policiesCache().getAsync(POLICY_ROOT + destination.getNamespace()).thenAccept(policies -> {
+            configCache.policiesCache().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> {
                 if (!policies.isPresent()) {
                     if (log.isDebugEnabled()) {
-                        log.debug("Policies node couldn't be found for destination : {}", destination);
+                        log.debug("Policies node couldn't be found for topic : {}", topicName);
                     }
                 } else {
                     if (isNotBlank(subscription) && !isSuperUser(role)) {
@@ -114,8 +114,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                         case Prefix:
                             if (!subscription.startsWith(role)) {
                                 PulsarServerException ex = new PulsarServerException(String.format(
-                                        "Failed to create consumer - The subscription name needs to be prefixed by the authentication role, like %s-xxxx for destination: %s",
-                                        role, destination));
+                                        "Failed to create consumer - The subscription name needs to be prefixed by the authentication role, like %s-xxxx for topic: %s",
+                                        role, topicName));
                                 permissionFuture.completeExceptionally(ex);
                                 return;
                             }
@@ -125,17 +125,17 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                         }
                     }
                 }
-                checkAuthorization(destination, role, AuthAction.consume).thenAccept(isAuthorized -> {
+                checkAuthorization(topicName, role, AuthAction.consume).thenAccept(isAuthorized -> {
                     permissionFuture.complete(isAuthorized);
                 });
             }).exceptionally(ex -> {
-                log.warn("Client with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+                log.warn("Client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
                         ex.getMessage());
                 permissionFuture.completeExceptionally(ex);
                 return null;
             });
         } catch (Exception e) {
-            log.warn("Client  with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+            log.warn("Client  with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
                     e.getMessage());
             permissionFuture.completeExceptionally(e);
         }
@@ -143,20 +143,20 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
     }
 
     /**
-     * Check whether the specified role can perform a lookup for the specified destination.
+     * Check whether the specified role can perform a lookup for the specified topic.
      *
      * For that the caller needs to have producer or consumer permission.
      *
-     * @param destination
+     * @param topicName
      * @param role
      * @return
      * @throws Exception
      */
     @Override
-    public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+    public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
             AuthenticationDataSource authenticationData) {
         CompletableFuture<Boolean> finalResult = new CompletableFuture<Boolean>();
-        canProduceAsync(destination, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
+        canProduceAsync(topicName, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
             if (ex == null) {
                 if (produceAuthorized) {
                     finalResult.complete(produceAuthorized);
@@ -165,11 +165,11 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
             } else {
                 if (log.isDebugEnabled()) {
                     log.debug(
-                            "Destination [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
-                            destination.toString(), role, ex.getMessage());
+                            "Topic [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
+                            topicName.toString(), role, ex.getMessage());
                 }
             }
-            canConsumeAsync(destination, role, authenticationData, null).whenComplete((consumeAuthorized, e) -> {
+            canConsumeAsync(topicName, role, authenticationData, null).whenComplete((consumeAuthorized, e) -> {
                 if (e == null) {
                     if (consumeAuthorized) {
                         finalResult.complete(consumeAuthorized);
@@ -178,8 +178,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                 } else {
                     if (log.isDebugEnabled()) {
                         log.debug(
-                                "Destination [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
-                                destination.toString(), role, e.getMessage());
+                                "Topic [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
+                                topicName.toString(), role, e.getMessage());
 
                     }
                     finalResult.completeExceptionally(e);
@@ -192,9 +192,9 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
     }
 
     @Override
-    public CompletableFuture<Void> grantPermissionAsync(DestinationName destination, Set<AuthAction> actions,
+    public CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<AuthAction> actions,
             String role, String authDataJson) {
-        return grantPermissionAsync(destination.getNamespaceObject(), actions, role, authDataJson);
+        return grantPermissionAsync(topicName.getNamespaceObject(), actions, role, authDataJson);
     }
 
     @Override
@@ -246,34 +246,34 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
         return result;
     }
 
-    private CompletableFuture<Boolean> checkAuthorization(DestinationName destination, String role, AuthAction action) {
+    private CompletableFuture<Boolean> checkAuthorization(TopicName topicName, String role, AuthAction action) {
         if (isSuperUser(role)) {
             return CompletableFuture.completedFuture(true);
         } else {
-            return checkPermission(destination, role, action)
-                    .thenApply(isPermission -> isPermission && checkCluster(destination));
+            return checkPermission(topicName, role, action)
+                    .thenApply(isPermission -> isPermission && checkCluster(topicName));
         }
     }
 
-    private boolean checkCluster(DestinationName destination) {
-        if (destination.isGlobal() || conf.getClusterName().equals(destination.getCluster())) {
+    private boolean checkCluster(TopicName topicName) {
+        if (topicName.isGlobal() || conf.getClusterName().equals(topicName.getCluster())) {
             return true;
         } else {
             if (log.isDebugEnabled()) {
-                log.debug("Destination [{}] does not belong to local cluster [{}]", destination.toString(),
+                log.debug("Topic [{}] does not belong to local cluster [{}]", topicName.toString(),
                         conf.getClusterName());
             }
             return false;
         }
     }
 
-    public CompletableFuture<Boolean> checkPermission(DestinationName destination, String role, AuthAction action) {
+    public CompletableFuture<Boolean> checkPermission(TopicName topicName, String role, AuthAction action) {
         CompletableFuture<Boolean> permissionFuture = new CompletableFuture<>();
         try {
-            configCache.policiesCache().getAsync(POLICY_ROOT + destination.getNamespace()).thenAccept(policies -> {
+            configCache.policiesCache().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> {
                 if (!policies.isPresent()) {
                     if (log.isDebugEnabled()) {
-                        log.debug("Policies node couldn't be found for destination : {}", destination);
+                        log.debug("Policies node couldn't be found for topic : {}", topicName);
                     }
                 } else {
                     Map<String, Set<AuthAction>> namespaceRoles = policies.get().auth_policies.namespace_auth;
@@ -284,13 +284,13 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                         return;
                     }
 
-                    Map<String, Set<AuthAction>> destinationRoles = policies.get().auth_policies.destination_auth
-                            .get(destination.toString());
-                    if (destinationRoles != null) {
-                        // Destination has custom policy
-                        Set<AuthAction> destinationActions = destinationRoles.get(role);
-                        if (destinationActions != null && destinationActions.contains(action)) {
-                            // The role has destination level permission
+                    Map<String, Set<AuthAction>> topicRoles = policies.get().auth_policies.destination_auth
+                            .get(topicName.toString());
+                    if (topicRoles != null) {
+                        // Topic has custom policy
+                        Set<AuthAction> topicActions = topicRoles.get(role);
+                        if (topicActions != null && topicActions.contains(action)) {
+                            // The role has topic level permission
                             permissionFuture.complete(true);
                             return;
                         }
@@ -304,8 +304,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                             return;
                         }
 
-                        if (destinationRoles != null && checkWildcardPermission(role, action, destinationRoles)) {
-                            // The role has destination level permission by wildcard match
+                        if (topicRoles != null && checkWildcardPermission(role, action, topicRoles)) {
+                            // The role has topic level permission by wildcard match
                             permissionFuture.complete(true);
                             return;
                         }
@@ -313,13 +313,13 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
                 }
                 permissionFuture.complete(false);
             }).exceptionally(ex -> {
-                log.warn("Client  with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+                log.warn("Client  with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
                         ex.getMessage());
                 permissionFuture.completeExceptionally(ex);
                 return null;
             });
         } catch (Exception e) {
-            log.warn("Client  with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+            log.warn("Client  with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
                     e.getMessage());
             permissionFuture.completeExceptionally(e);
         }
@@ -352,7 +352,7 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
      * Super user roles are allowed to do anything, used for replication primarily
      *
      * @param role
-     *            the app id used to receive messages from the destination.
+     *            the app id used to receive messages from the topic.
      */
     public boolean isSuperUser(String role) {
         Set<String> superUserRoles = conf.getSuperUserRoles();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
index bec354b..5ee1532 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.web.WebService;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
 import org.apache.pulsar.common.configuration.VipStatus;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.ClusterData;
@@ -459,13 +459,13 @@ public class PulsarService implements AutoCloseable {
     }
 
     /**
-     * Load all the destination contained in a namespace
+     * Load all the topics contained in a namespace
      *
      * @param bundle
      *            <code>NamespaceBundle</code> to identify the service unit
      * @throws Exception
      */
-    public void loadNamespaceDestinations(NamespaceBundle bundle) {
+    public void loadNamespaceTopics(NamespaceBundle bundle) {
         executor.submit(() -> {
             LOG.info("Loading all topics on bundle: {}", bundle);
 
@@ -473,10 +473,10 @@ public class PulsarService implements AutoCloseable {
             List<CompletableFuture<Topic>> persistentTopics = Lists.newArrayList();
             long topicLoadStart = System.nanoTime();
 
-            for (String topic : getNamespaceService().getListOfDestinations(nsName)) {
+            for (String topic : getNamespaceService().getListOfTopics(nsName)) {
                 try {
-                    DestinationName dn = DestinationName.get(topic);
-                    if (bundle.includes(dn)) {
+                    TopicName topicName = TopicName.get(topic);
+                    if (bundle.includes(topicName)) {
                         CompletableFuture<Topic> future = brokerService.getTopic(topic);
                         if (future != null) {
                             persistentTopics.add(future);
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 b91c63a..527f455 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
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
 import org.apache.pulsar.broker.web.PulsarWebResource;
 import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -103,7 +103,7 @@ public abstract class AdminResource extends PulsarWebResource {
     }
 
     /**
-     * Get the domain of the destination (whether it's queue or topic)
+     * Get the domain of the topic (whether it's persistent or non-persistent)
      */
     protected String domain() {
         if (uri.getPath().startsWith("persistent/")) {
@@ -237,28 +237,28 @@ public abstract class AdminResource extends PulsarWebResource {
         }
     }
 
-    protected DestinationName destinationName;
+    protected TopicName topicName;
 
-    protected void validateDestinationName(String property, String namespace, String encodedTopic) {
+    protected void validateTopicName(String property, String namespace, String encodedTopic) {
         String topic = Codec.decode(encodedTopic);
         try {
             this.namespaceName = NamespaceName.get(property, namespace);
-            this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+            this.topicName = TopicName.get(domain(), namespaceName, topic);
         } catch (IllegalArgumentException e) {
             log.warn("[{}] Failed to validate topic name {}://{}/{}/{}", clientAppId(), domain(), property, namespace,
                     topic, e);
             throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid");
         }
 
-        this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+        this.topicName = TopicName.get(domain(), namespaceName, topic);
     }
 
     @Deprecated
-    protected void validateDestinationName(String property, String cluster, String namespace, String encodedTopic) {
+    protected void validateTopicName(String property, String cluster, String namespace, String encodedTopic) {
         String topic = Codec.decode(encodedTopic);
         try {
             this.namespaceName = NamespaceName.get(property, cluster, namespace);
-            this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+            this.topicName = TopicName.get(domain(), namespaceName, topic);
         } catch (IllegalArgumentException e) {
             log.warn("[{}] Failed to validate topic name {}://{}/{}/{}/{}", clientAppId(), domain(), property, cluster,
                     namespace, topic, e);
@@ -358,30 +358,30 @@ public abstract class AdminResource extends PulsarWebResource {
         return pulsar().getConfigurationCache().failureDomainListCache();
     }
 
-    protected PartitionedTopicMetadata getPartitionedTopicMetadata(DestinationName destinationName,
+    protected PartitionedTopicMetadata getPartitionedTopicMetadata(TopicName topicName,
             boolean authoritative) {
-        validateClusterOwnership(destinationName.getCluster());
+        validateClusterOwnership(topicName.getCluster());
         // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
         // serve/redirect request else fail partitioned-metadata-request so, client fails while creating
         // producer/consumer
-        validateGlobalNamespaceOwnership(destinationName.getNamespaceObject());
+        validateGlobalNamespaceOwnership(topicName.getNamespaceObject());
 
         try {
-            checkConnect(destinationName);
+            checkConnect(topicName);
         } catch (WebApplicationException e) {
-            validateAdminAccessOnProperty(destinationName.getProperty());
+            validateAdminAccessOnProperty(topicName.getProperty());
         } catch (Exception e) {
             // unknown error marked as internal server error
-            log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destinationName,
+            log.warn("Unexpected error while authorizing lookup. topic={}, role={}. Error: {}", topicName,
                     clientAppId(), e.getMessage(), e);
             throw new RestException(e);
         }
 
-        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), destinationName.getEncodedLocalName());
+        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), topicName.getEncodedLocalName());
         PartitionedTopicMetadata partitionMetadata = fetchPartitionedTopicMetadata(pulsar(), path);
 
         if (log.isDebugEnabled()) {
-            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), destinationName,
+            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), topicName,
                     partitionMetadata.partitions);
         }
         return partitionMetadata;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
index 549a033..9da3fc6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
@@ -87,12 +87,12 @@ public class BrokerStatsBase extends AdminResource {
 
     @GET
     @Path("/destinations")
-    @ApiOperation(value = "Get all the destination stats by namesapce", response = OutputStream.class, responseContainer = "OutputStream") // https://github.com/swagger-api/swagger-ui/issues/558
+    @ApiOperation(value = "Get all the topic stats by namesapce", response = OutputStream.class, responseContainer = "OutputStream") // https://github.com/swagger-api/swagger-ui/issues/558
                                                                                                                                            // map
                                                                                                                                            // support
                                                                                                                                            // missing
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
-    public StreamingOutput getDestinations2() throws Exception {
+    public StreamingOutput getTopics2() throws Exception {
         // Ensure super user access only
         validateSuperUserAccess();
         return output -> pulsar().getBrokerService().getDimensionMetrics(statsBuf -> {
@@ -138,14 +138,14 @@ public class BrokerStatsBase extends AdminResource {
         try {
             return BookieClientStatsGenerator.generate(pulsar());
         } catch (Exception e) {
-            log.error("[{}] Failed to generate pending bookie ops stats for destinations", clientAppId(), e);
+            log.error("[{}] Failed to generate pending bookie ops stats for topicss", clientAppId(), e);
             throw new RestException(e);
         }
     }
 
     @GET
     @Path("/load-report")
-    @ApiOperation(value = "Get Load for this broker", notes = "consists of destinationstats & systemResourceUsage", response = LoadReport.class)
+    @ApiOperation(value = "Get Load for this broker", notes = "consists of topics stats & systemResourceUsage", response = LoadReport.class)
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public LoadManagerReport getLoadReport() throws Exception {
         // Ensure super user access only
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 0dda3ae..902a56f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -170,13 +170,13 @@ public abstract class NamespacesBase extends AdminResource {
 
         boolean isEmpty;
         try {
-            isEmpty = pulsar().getNamespaceService().getListOfDestinations(namespaceName).isEmpty();
+            isEmpty = pulsar().getNamespaceService().getListOfTopics(namespaceName).isEmpty();
         } catch (Exception e) {
             throw new RestException(e);
         }
 
         if (!isEmpty) {
-            log.debug("Found destinations on namespace {}", namespaceName);
+            log.debug("Found topics on namespace {}", namespaceName);
             throw new RestException(Status.CONFLICT, "Cannot delete non empty namespace");
         }
 
@@ -269,11 +269,11 @@ public abstract class NamespacesBase extends AdminResource {
         NamespaceBundle bundle = validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange,
                 authoritative, true);
         try {
-            List<String> destinations = pulsar().getNamespaceService().getListOfDestinations(namespaceName);
-            for (String destination : destinations) {
-                NamespaceBundle destinationBundle = (NamespaceBundle) pulsar().getNamespaceService()
-                        .getBundle(DestinationName.get(destination));
-                if (bundle.equals(destinationBundle)) {
+            List<String> topics = pulsar().getNamespaceService().getListOfTopics(namespaceName);
+            for (String topic : topics) {
+                NamespaceBundle topicBundle = (NamespaceBundle) pulsar().getNamespaceService()
+                        .getBundle(TopicName.get(topic));
+                if (bundle.equals(topicBundle)) {
                     throw new RestException(Status.CONFLICT, "Cannot delete non empty bundle");
                 }
             }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
index 5d83f8d..9386df5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
@@ -77,8 +77,8 @@ import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
 import org.apache.pulsar.common.compression.CompressionCodec;
 import org.apache.pulsar.common.compression.CompressionCodecProvider;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.AuthPolicies;
@@ -126,24 +126,24 @@ public class PersistentTopicsBase extends AdminResource {
             throw new RestException(e);
         }
 
-        List<String> destinations = Lists.newArrayList();
+        List<String> topics = Lists.newArrayList();
 
         try {
             String path = String.format("/managed-ledgers/%s/%s", namespaceName.toString(), domain());
-            for (String destination : managedLedgerListCache().get(path)) {
-                if (domain().equals(DestinationDomain.persistent.toString())) {
-                    destinations.add(DestinationName.get(domain(), namespaceName, decode(destination)).toString());
+            for (String topic : managedLedgerListCache().get(path)) {
+                if (domain().equals(TopicDomain.persistent.toString())) {
+                    topics.add(TopicName.get(domain(), namespaceName, decode(topic)).toString());
                 }
             }
         } catch (KeeperException.NoNodeException e) {
-            // NoNode means there are no destination in this domain for this namespace
+            // NoNode means there are no topics in this domain for this namespace
         } catch (Exception e) {
-            log.error("[{}] Failed to get destination list for namespace {}", clientAppId(), namespaceName, e);
+            log.error("[{}] Failed to get topics list for namespace {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
 
-        destinations.sort(null);
-        return destinations;
+        topics.sort(null);
+        return topics;
     }
 
     protected List<String> internalGetPartitionedTopicList() {
@@ -165,8 +165,8 @@ public class PersistentTopicsBase extends AdminResource {
 
         try {
             String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain());
-            List<String> destinations = globalZk().getChildren(partitionedTopicPath, false);
-            partitionedTopics = destinations.stream()
+            List<String> topics = globalZk().getChildren(partitionedTopicPath, false);
+            partitionedTopics = topics.stream()
                     .map(s -> String.format("persistent://%s/%s", namespaceName.toString(), decode(s)))
                     .collect(Collectors.toList());
         } catch (KeeperException.NoNodeException e) {
@@ -181,11 +181,11 @@ public class PersistentTopicsBase extends AdminResource {
         return partitionedTopics;
     }
 
-    protected Map<String, Set<AuthAction>> internalGetPermissionsOnDestination() {
+    protected Map<String, Set<AuthAction>> internalGetPermissionsOnTopic() {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessOnProperty(namespaceName.getProperty());
 
-        String destinationUri = destinationName.toString();
+        String topicUri = topicName.toString();
 
         try {
             Policies policies = policiesCache().get(path(POLICIES, namespaceName.toString()))
@@ -199,17 +199,17 @@ public class PersistentTopicsBase extends AdminResource {
                 permissions.put(role, auth.namespace_auth.get(role));
             }
 
-            // Then add destination level permissions
-            if (auth.destination_auth.containsKey(destinationUri)) {
-                for (Map.Entry<String, Set<AuthAction>> entry : auth.destination_auth.get(destinationUri).entrySet()) {
+            // Then add topic level permissions
+            if (auth.destination_auth.containsKey(topicUri)) {
+                for (Map.Entry<String, Set<AuthAction>> entry : auth.destination_auth.get(topicUri).entrySet()) {
                     String role = entry.getKey();
-                    Set<AuthAction> destinationPermissions = entry.getValue();
+                    Set<AuthAction> topicPermissions = entry.getValue();
 
                     if (!permissions.containsKey(role)) {
-                        permissions.put(role, destinationPermissions);
+                        permissions.put(role, topicPermissions);
                     } else {
-                        // Do the union between namespace and destination level
-                        Set<AuthAction> union = Sets.union(permissions.get(role), destinationPermissions);
+                        // Do the union between namespace and topic level
+                        Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
                         permissions.put(role, union);
                     }
                 }
@@ -217,50 +217,50 @@ public class PersistentTopicsBase extends AdminResource {
 
             return permissions;
         } catch (Exception e) {
-            log.error("[{}] Failed to get permissions for destination {}", clientAppId(), destinationUri, e);
+            log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicUri, e);
             throw new RestException(e);
         }
     }
 
     protected void validateAdminAndClientPermission() {
         try {
-            validateAdminAccessOnProperty(destinationName.getProperty());
+            validateAdminAccessOnProperty(topicName.getProperty());
         } catch (Exception ve) {
             try {
-                checkAuthorization(pulsar(), destinationName, clientAppId(), clientAuthData());
+                checkAuthorization(pulsar(), topicName, clientAppId(), clientAuthData());
             } catch (RestException re) {
                 throw re;
             } catch (Exception e) {
                 // unknown error marked as internal server error
-                log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}",
-                        destinationName, clientAppId(), e.getMessage(), e);
+                log.warn("Unexpected error while authorizing request. topic={}, role={}. Error: {}",
+                        topicName, clientAppId(), e.getMessage(), e);
                 throw new RestException(e);
             }
         }
     }
 
-    public void validateAdminOperationOnDestination(boolean authoritative) {
-        validateAdminAccessOnProperty(destinationName.getProperty());
-        validateDestinationOwnership(destinationName, authoritative);
+    public void validateAdminOperationOnTopic(boolean authoritative) {
+        validateAdminAccessOnProperty(topicName.getProperty());
+        validateTopicOwnership(topicName, authoritative);
     }
 
-    protected void internalGrantPermissionsOnDestination(String role, Set<AuthAction> actions) {
+    protected void internalGrantPermissionsOnTopic(String role, Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessOnProperty(namespaceName.getProperty());
         validatePoliciesReadOnlyAccess();
 
-        String destinationUri = destinationName.toString();
+        String topicUri = topicName.toString();
 
         try {
             Stat nodeStat = new Stat();
             byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
             Policies policies = jsonMapper().readValue(content, Policies.class);
 
-            if (!policies.auth_policies.destination_auth.containsKey(destinationUri)) {
-                policies.auth_policies.destination_auth.put(destinationUri, new TreeMap<String, Set<AuthAction>>());
+            if (!policies.auth_policies.destination_auth.containsKey(topicUri)) {
+                policies.auth_policies.destination_auth.put(topicUri, new TreeMap<String, Set<AuthAction>>());
             }
 
-            policies.auth_policies.destination_auth.get(destinationUri).put(role, actions);
+            policies.auth_policies.destination_auth.get(topicUri).put(role, actions);
 
             // Write the new policies to zookeeper
             globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies),
@@ -269,25 +269,25 @@ public class PersistentTopicsBase extends AdminResource {
             // invalidate the local cache to force update
             policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
 
-            log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions,
-                    destinationUri);
+            log.info("[{}] Successfully granted access for role {}: {} - topic {}", clientAppId(), role, actions,
+                    topicUri);
 
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to grant permissions on destination {}: Namespace does not exist", clientAppId(),
-                    destinationUri);
+            log.warn("[{}] Failed to grant permissions on topic {}: Namespace does not exist", clientAppId(),
+                    topicUri);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to grant permissions for destination {}", clientAppId(), destinationUri, e);
+            log.error("[{}] Failed to grant permissions for topic {}", clientAppId(), topicUri, e);
             throw new RestException(e);
         }
     }
 
-    protected void internalRevokePermissionsOnDestination(String role) {
+    protected void internalRevokePermissionsOnTopic(String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessOnProperty(namespaceName.getProperty());
         validatePoliciesReadOnlyAccess();
 
-        String destinationUri = destinationName.toString();
+        String topicUri = topicName.toString();
         Stat nodeStat = new Stat();
         Policies policies;
 
@@ -295,22 +295,22 @@ public class PersistentTopicsBase extends AdminResource {
             byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
             policies = jsonMapper().readValue(content, Policies.class);
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to revoke permissions on destination {}: Namespace does not exist", clientAppId(),
-                    destinationUri);
+            log.warn("[{}] Failed to revoke permissions on topic {}: Namespace does not exist", clientAppId(),
+                    topicUri);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e);
+            log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e);
             throw new RestException(e);
         }
 
-        if (!policies.auth_policies.destination_auth.containsKey(destinationUri)
-                || !policies.auth_policies.destination_auth.get(destinationUri).containsKey(role)) {
-            log.warn("[{}] Failed to revoke permission from role {} on destination: Not set at destination level",
-                    clientAppId(), role, destinationUri);
-            throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the destination level");
+        if (!policies.auth_policies.destination_auth.containsKey(topicUri)
+                || !policies.auth_policies.destination_auth.get(topicUri).containsKey(role)) {
+            log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level",
+                    clientAppId(), role, topicUri);
+            throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the topic level");
         }
 
-        policies.auth_policies.destination_auth.get(destinationUri).remove(role);
+        policies.auth_policies.destination_auth.get(topicUri).remove(role);
 
         try {
             // Write the new policies to zookeeper
@@ -321,32 +321,32 @@ public class PersistentTopicsBase extends AdminResource {
             policiesCache().invalidate(namespacePath);
             globalZkCache().invalidate(namespacePath);
 
-            log.info("[{}] Successfully revoke access for role {} - destination {}", clientAppId(), role,
-                    destinationUri);
+            log.info("[{}] Successfully revoke access for role {} - topic {}", clientAppId(), role,
+                    topicUri);
         } catch (Exception e) {
-            log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e);
+            log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e);
             throw new RestException(e);
         }
     }
 
     protected void internalCreatePartitionedTopic(int numPartitions, boolean authoritative) {
-        validateAdminAccessOnProperty(destinationName.getProperty());
+        validateAdminAccessOnProperty(topicName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
             String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
-                    destinationName.getEncodedLocalName());
+                    topicName.getEncodedLocalName());
             byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
             zkCreateOptimistic(path, data);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
         } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
             throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
             throw new RestException(e);
         }
     }
@@ -363,28 +363,28 @@ public class PersistentTopicsBase extends AdminResource {
      * @param numPartitions
      */
     protected void internalUpdatePartitionedTopic(int numPartitions) {
-        validateAdminAccessOnProperty(destinationName.getProperty());
-        if (destinationName.isGlobal()) {
+        validateAdminAccessOnProperty(topicName.getProperty());
+        if (topicName.isGlobal()) {
             log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(),
-                    destinationName);
+                    topicName);
             throw new RestException(Status.FORBIDDEN, "Update forbidden on global namespace");
         }
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
-            updatePartitionedTopic(destinationName, numPartitions).get();
+            updatePartitionedTopic(topicName, numPartitions).get();
         } catch (Exception e) {
             if (e.getCause() instanceof RestException) {
                 throw (RestException) e.getCause();
             }
-            log.error("[{}] Failed to update partitioned topic {}", clientAppId(), destinationName, e.getCause());
+            log.error("[{}] Failed to update partitioned topic {}", clientAppId(), topicName, e.getCause());
             throw new RestException(e.getCause());
         }
     }
 
     protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean authoritative) {
-        PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (metadata.partitions > 1) {
             validateClientVersion();
         }
@@ -392,16 +392,16 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalDeletePartitionedTopic(boolean authoritative) {
-        validateAdminAccessOnProperty(destinationName.getProperty());
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        validateAdminAccessOnProperty(topicName.getProperty());
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         int numPartitions = partitionMetadata.partitions;
         if (numPartitions > 0) {
             final CompletableFuture<Void> future = new CompletableFuture<>();
             final AtomicInteger count = new AtomicInteger(numPartitions);
             try {
                 for (int i = 0; i < numPartitions; i++) {
-                    DestinationName dn_partition = destinationName.getPartition(i);
-                    pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString())
+                    TopicName topicNamePartition = topicName.getPartition(i);
+                    pulsar().getAdminClient().persistentTopics().deleteAsync(topicNamePartition.toString())
                             .whenComplete((r, ex) -> {
                                 if (ex != null) {
                                     if (ex instanceof NotFoundException) {
@@ -410,16 +410,16 @@ public class PersistentTopicsBase extends AdminResource {
                                         // For all other exception, we fail the delete partition method even if a single
                                         // partition is failed to be deleted
                                         if (log.isDebugEnabled()) {
-                                            log.debug("[{}] Partition not found: {}", clientAppId(), dn_partition);
+                                            log.debug("[{}] Partition not found: {}", clientAppId(), topicNamePartition);
                                         }
                                     } else {
                                         future.completeExceptionally(ex);
-                                        log.error("[{}] Failed to delete partition {}", clientAppId(), dn_partition,
+                                        log.error("[{}] Failed to delete partition {}", clientAppId(), topicNamePartition,
                                                 ex);
                                         return;
                                     }
                                 } else {
-                                    log.info("[{}] Deleted partition {}", clientAppId(), dn_partition);
+                                    log.info("[{}] Deleted partition {}", clientAppId(), topicNamePartition);
                                 }
                                 if (count.decrementAndGet() == 0) {
                                     future.complete(null);
@@ -439,44 +439,44 @@ public class PersistentTopicsBase extends AdminResource {
 
         // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted
         String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
-                destinationName.getEncodedLocalName());
+                topicName.getEncodedLocalName());
         try {
             globalZk().delete(path, -1);
             globalZkCache().invalidate(path);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Deleted partitioned topic {}", clientAppId(), destinationName);
+            log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName);
         } catch (KeeperException.NoNodeException nne) {
             throw new RestException(Status.NOT_FOUND, "Partitioned topic does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), destinationName, e);
+            log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e);
             throw new RestException(e);
         }
     }
 
     protected void internalUnloadTopic(boolean authoritative) {
-        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
-        if (destinationName.isGlobal()) {
+        log.info("[{}] Unloading topic {}", clientAppId(), topicName);
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        unloadTopic(destinationName, authoritative);
+        unloadTopic(topicName, authoritative);
     }
 
     protected void internalDeleteTopic(boolean authoritative) {
-        validateAdminOperationOnDestination(authoritative);
-        Topic topic = getTopicReference(destinationName);
-        if (destinationName.isGlobal()) {
+        validateAdminOperationOnTopic(authoritative);
+        Topic topic = getTopicReference(topicName);
+        if (topicName.isGlobal()) {
             // Delete is disallowed on global topic
-            log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), destinationName);
+            log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), topicName);
             throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace");
         }
 
         try {
             topic.delete().get();
-            log.info("[{}] Successfully removed topic {}", clientAppId(), destinationName);
+            log.info("[{}] Successfully removed topic {}", clientAppId(), topicName);
         } catch (Exception e) {
             Throwable t = e.getCause();
-            log.error("[{}] Failed to get delete topic {}", clientAppId(), destinationName, t);
+            log.error("[{}] Failed to get delete topic {}", clientAppId(), topicName, t);
             if (t instanceof TopicBusyException) {
                 throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions");
             } else {
@@ -486,29 +486,29 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected List<String> internalGetSubscriptions(boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
 
         List<String> subscriptions = Lists.newArrayList();
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 // get the subscriptions only from the 1st partition since all the other partitions will have the same
                 // subscriptions
                 subscriptions.addAll(pulsar().getAdminClient().persistentTopics()
-                        .getSubscriptions(destinationName.getPartition(0).toString()));
+                        .getSubscriptions(topicName.getPartition(0).toString()));
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
-            validateAdminOperationOnDestination(authoritative);
-            Topic topic = getTopicReference(destinationName);
+            validateAdminOperationOnTopic(authoritative);
+            Topic topic = getTopicReference(topicName);
 
             try {
                 topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName));
             } catch (Exception e) {
-                log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), destinationName);
+                log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName);
                 throw new RestException(e);
             }
         }
@@ -518,30 +518,30 @@ public class PersistentTopicsBase extends AdminResource {
 
     protected PersistentTopicStats internalGetStats(boolean authoritative) {
         validateAdminAndClientPermission();
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        validateDestinationOwnership(destinationName, authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicOwnership(topicName, authoritative);
+        Topic topic = getTopicReference(topicName);
         return topic.getStats();
     }
 
     protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) {
         validateAdminAndClientPermission();
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        validateDestinationOwnership(destinationName, authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicOwnership(topicName, authoritative);
+        Topic topic = getTopicReference(topicName);
         return topic.getInternalStats();
     }
 
     protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse) {
-        validateAdminAccessOnProperty(destinationName.getProperty());
-        if (destinationName.isGlobal()) {
+        validateAdminAccessOnProperty(topicName.getProperty());
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        String managedLedger = destinationName.getPersistenceNamingEncoding();
+        String managedLedger = topicName.getPersistenceNamingEncoding();
         pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() {
             @Override
             public void getInfoComplete(ManagedLedgerInfo info, Object ctx) {
@@ -558,20 +558,20 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected PartitionedTopicStats internalGetPartitionedStats(boolean authoritative) {
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions == 0) {
             throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found");
         }
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
         PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata);
         try {
             for (int i = 0; i < partitionMetadata.partitions; i++) {
                 PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics()
-                        .getStats(destinationName.getPartition(i).toString());
+                        .getStats(topicName.getPartition(i).toString());
                 stats.add(partitionStats);
-                stats.partitions.put(destinationName.getPartition(i).toString(), partitionStats);
+                stats.partitions.put(topicName.getPartition(i).toString(), partitionStats);
             }
         } catch (Exception e) {
             throw new RestException(e);
@@ -580,15 +580,15 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalDeleteSubscription(String subName, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
                     pulsar().getAdminClient().persistentTopics()
-                            .deleteSubscription(destinationName.getPartition(i).toString(), subName);
+                            .deleteSubscription(topicName.getPartition(i).toString(), subName);
                 }
             } catch (Exception e) {
                 if (e instanceof NotFoundException) {
@@ -596,18 +596,18 @@ public class PersistentTopicsBase extends AdminResource {
                 } else if (e instanceof PreconditionFailedException) {
                     throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
                 } else {
-                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
+                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e);
                     throw new RestException(e);
                 }
             }
         } else {
-            validateAdminOperationOnDestination(authoritative);
-            Topic topic = getTopicReference(destinationName);
+            validateAdminOperationOnTopic(authoritative);
+            Topic topic = getTopicReference(topicName);
             try {
                 Subscription sub = topic.getSubscription(subName);
                 checkNotNull(sub);
                 sub.delete().get();
-                log.info("[{}][{}] Deleted subscription {}", clientAppId(), destinationName, subName);
+                log.info("[{}][{}] Deleted subscription {}", clientAppId(), topicName, subName);
             } catch (Exception e) {
                 Throwable t = e.getCause();
                 if (e instanceof NullPointerException) {
@@ -615,7 +615,7 @@ public class PersistentTopicsBase extends AdminResource {
                 } else if (t instanceof SubscriptionBusyException) {
                     throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
                 } else {
-                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
+                    log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e);
                     throw new RestException(t);
                 }
             }
@@ -623,22 +623,22 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalSkipAllMessages(String subName, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
                     pulsar().getAdminClient().persistentTopics()
-                            .skipAllMessages(destinationName.getPartition(i).toString(), subName);
+                            .skipAllMessages(topicName.getPartition(i).toString(), subName);
                 }
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
-            validateAdminOperationOnDestination(authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+            validateAdminOperationOnTopic(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
             try {
                 if (subName.startsWith(topic.replicatorPrefix)) {
                     String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -650,26 +650,26 @@ public class PersistentTopicsBase extends AdminResource {
                     checkNotNull(sub);
                     sub.clearBacklog().get();
                 }
-                log.info("[{}] Cleared backlog on {} {}", clientAppId(), destinationName, subName);
+                log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName);
             } catch (NullPointerException npe) {
                 throw new RestException(Status.NOT_FOUND, "Subscription not found");
             } catch (Exception exception) {
-                log.error("[{}] Failed to skip all messages {} {}", clientAppId(), destinationName, subName, exception);
+                log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, exception);
                 throw new RestException(exception);
             }
         }
     }
 
     protected void internalSkipMessages(String subName, int numMessages, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Skip messages on a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(authoritative);
-        PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+        validateAdminOperationOnTopic(authoritative);
+        PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
         try {
             if (subName.startsWith(topic.replicatorPrefix)) {
                 String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -681,37 +681,37 @@ public class PersistentTopicsBase extends AdminResource {
                 checkNotNull(sub);
                 sub.skipMessages(numMessages).get();
             }
-            log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, destinationName, subName);
+            log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, topicName, subName);
         } catch (NullPointerException npe) {
             throw new RestException(Status.NOT_FOUND, "Subscription not found");
         } catch (Exception exception) {
-            log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, destinationName, subName,
+            log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName,
                     exception);
             throw new RestException(exception);
         }
     }
 
     protected void internalExpireMessagesForAllSubscriptions(int expireTimeInSeconds, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             try {
-                // expire messages for each partition destination
+                // expire messages for each partition topic
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
                     pulsar().getAdminClient().persistentTopics().expireMessagesForAllSubscriptions(
-                            destinationName.getPartition(i).toString(), expireTimeInSeconds);
+                            topicName.getPartition(i).toString(), expireTimeInSeconds);
                 }
             } catch (Exception e) {
                 log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds,
-                        destinationName, e);
+                        topicName, e);
                 throw new RestException(e);
             }
         } else {
             // validate ownership and redirect if current broker is not owner
-            validateAdminOperationOnDestination(authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+            validateAdminOperationOnTopic(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
             topic.getReplicators().forEach((subName, replicator) -> {
                 internalExpireMessages(subName, expireTimeInSeconds, authoritative);
             });
@@ -722,10 +722,10 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalResetCursor(String subName, long timestamp, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
 
         if (partitionMetadata.partitions > 0) {
             int numParts = partitionMetadata.partitions;
@@ -733,7 +733,7 @@ public class PersistentTopicsBase extends AdminResource {
             Exception partitionException = null;
             try {
                 for (int i = 0; i < numParts; i++) {
-                    pulsar().getAdminClient().persistentTopics().resetCursor(destinationName.getPartition(i).toString(),
+                    pulsar().getAdminClient().persistentTopics().resetCursor(topicName.getPartition(i).toString(),
                             subName, timestamp);
                 }
             } catch (PreconditionFailedException pfe) {
@@ -743,24 +743,24 @@ public class PersistentTopicsBase extends AdminResource {
                 partitionException = pfe;
             } catch (Exception e) {
                 log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
-                        destinationName, subName, timestamp, e);
+                        topicName, subName, timestamp, e);
                 throw new RestException(e);
             }
             // report an error to user if unable to reset for all partitions
             if (numPartException == numParts) {
                 log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
-                        destinationName, subName, timestamp, partitionException);
+                        topicName, subName, timestamp, partitionException);
                 throw new RestException(Status.PRECONDITION_FAILED, partitionException.getMessage());
             } else if (numPartException > 0) {
                 log.warn("[{}][{}] partial errors for reset cursor on subscription {} to time {} - ", clientAppId(),
-                        destinationName, subName, timestamp, partitionException);
+                        topicName, subName, timestamp, partitionException);
             }
 
         } else {
-            validateAdminOperationOnDestination(authoritative);
-            log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destinationName,
+            validateAdminOperationOnTopic(authoritative);
+            log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), topicName,
                     subName, timestamp);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
             if (topic == null) {
                 throw new RestException(Status.NOT_FOUND, "Topic not found");
             }
@@ -768,12 +768,12 @@ public class PersistentTopicsBase extends AdminResource {
                 PersistentSubscription sub = topic.getSubscription(subName);
                 checkNotNull(sub);
                 sub.resetCursor(timestamp).get();
-                log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), destinationName, subName,
+                log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), topicName, subName,
                         timestamp);
             } catch (Exception e) {
                 Throwable t = e.getCause();
                 log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
-                        destinationName, subName, timestamp, e);
+                        topicName, subName, timestamp, e);
                 if (e instanceof NullPointerException) {
                     throw new RestException(Status.NOT_FOUND, "Subscription not found");
                 } else if (e instanceof NotAllowedException) {
@@ -789,13 +789,13 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalCreateSubscription(String subscriptionName, MessageIdImpl messageId, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destinationName,
+        log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), topicName,
                 subscriptionName, messageId);
 
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
 
         try {
             if (partitionMetadata.partitions > 0) {
@@ -805,15 +805,15 @@ public class PersistentTopicsBase extends AdminResource {
 
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
                     futures.add(admin.persistentTopics().createSubscriptionAsync(
-                            destinationName.getPartition(i).toString(),
+                            topicName.getPartition(i).toString(),
                             subscriptionName, messageId));
                 }
 
                 FutureUtil.waitForAll(futures).join();
             } else {
-                validateAdminOperationOnDestination(authoritative);
+                validateAdminOperationOnTopic(authoritative);
 
-                PersistentTopic topic = (PersistentTopic) getOrCreateTopic(destinationName);
+                PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName);
 
                 if (topic.getSubscriptions().containsKey(subscriptionName)) {
                     throw new RestException(Status.CONFLICT, "Subscription already exists for topic");
@@ -823,12 +823,12 @@ public class PersistentTopicsBase extends AdminResource {
                         .createSubscription(subscriptionName).get();
                 subscription.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
                 log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(),
-                        destinationName, subscriptionName, messageId);
+                        topicName, subscriptionName, messageId);
             }
         } catch (Exception e) {
             Throwable t = e.getCause();
             log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(),
-                    destinationName, subscriptionName, messageId, e);
+                    topicName, subscriptionName, messageId, e);
             if (t instanceof SubscriptionInvalidCursorPosition) {
                 throw new RestException(Status.PRECONDITION_FAILED,
                         "Unable to find position for position specified: " + t.getMessage());
@@ -839,22 +839,22 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected void internalResetCursorOnPosition(String subName, boolean authoritative, MessageIdImpl messageId) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destinationName,
+        log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), topicName,
                 subName, messageId);
 
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
 
         if (partitionMetadata.partitions > 0) {
-            log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), destinationName,
+            log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), topicName,
                     subName);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
                     "Reset-cursor at position is not allowed for partitioned-topic");
         } else {
-            validateAdminOperationOnDestination(authoritative);
-            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+            validateAdminOperationOnTopic(authoritative);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
             if (topic == null) {
                 throw new RestException(Status.NOT_FOUND, "Topic not found");
             }
@@ -863,11 +863,11 @@ public class PersistentTopicsBase extends AdminResource {
                 checkNotNull(sub);
                 sub.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
                 log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(),
-                        destinationName, subName, messageId);
+                        topicName, subName, messageId);
             } catch (Exception e) {
                 Throwable t = e.getCause();
                 log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(),
-                        destinationName, subName, messageId, e);
+                        topicName, subName, messageId, e);
                 if (e instanceof NullPointerException) {
                     throw new RestException(Status.NOT_FOUND, "Subscription not found");
                 } else if (t instanceof SubscriptionInvalidCursorPosition) {
@@ -881,21 +881,21 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected Response internalPeekNthMessage(String subName, int messagePosition, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Peek messages on a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(authoritative);
-        if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
-            log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+        validateAdminOperationOnTopic(authoritative);
+        if (!(getTopicReference(topicName) instanceof PersistentTopic)) {
+            log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), topicName,
                     subName);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
                     "Skip messages on a non-persistent topic is not allowed");
         }
-        PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+        PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
         PersistentReplicator repl = null;
         PersistentSubscription sub = null;
         Entry entry = null;
@@ -956,7 +956,7 @@ public class PersistentTopicsBase extends AdminResource {
             throw new RestException(Status.NOT_FOUND, "Message not found");
         } catch (Exception exception) {
             log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition,
-                    destinationName, subName, exception);
+                    topicName, subName, exception);
             throw new RestException(exception);
         } finally {
             if (entry != null) {
@@ -966,11 +966,11 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
         // Validate that namespace exists, throw 404 if it doesn't exist
-        // note that we do not want to load the topic and hence skip validateAdminOperationOnDestination()
+        // note that we do not want to load the topic and hence skip validateAdminOperationOnTopic()
         try {
             policiesCache().get(path(POLICIES, namespaceName.toString()));
         } catch (KeeperException.NoNodeException e) {
@@ -984,7 +984,7 @@ public class PersistentTopicsBase extends AdminResource {
         PersistentOfflineTopicStats offlineTopicStats = null;
         try {
 
-            offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(destinationName);
+            offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(topicName);
             if (offlineTopicStats != null) {
                 // offline topic stat has a cost - so use cached value until TTL
                 long elapsedMs = System.currentTimeMillis() - offlineTopicStats.statGeneratedAt.getTime();
@@ -992,13 +992,13 @@ public class PersistentTopicsBase extends AdminResource {
                     return offlineTopicStats;
                 }
             }
-            final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(destinationName)
+            final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(topicName)
                     .get();
             ManagedLedgerOfflineBacklog offlineTopicBacklog = new ManagedLedgerOfflineBacklog(config.getDigestType(),
                     config.getPassword(), pulsar().getAdvertisedAddress(), false);
             offlineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog(
-                    (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), destinationName);
-            pulsar().getBrokerService().cacheOfflineTopicStats(destinationName, offlineTopicStats);
+                    (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), topicName);
+            pulsar().getBrokerService().cacheOfflineTopicStats(topicName, offlineTopicStats);
         } catch (Exception exception) {
             throw new RestException(exception);
         }
@@ -1006,48 +1006,48 @@ public class PersistentTopicsBase extends AdminResource {
     }
 
     protected MessageId internalTerminate(boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
             throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a partitioned topic is not allowed");
         }
-        validateAdminOperationOnDestination(authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateAdminOperationOnTopic(authoritative);
+        Topic topic = getTopicReference(topicName);
         try {
             return ((PersistentTopic) topic).terminate().get();
         } catch (Exception exception) {
-            log.error("[{}] Failed to terminated topic {}", clientAppId(), destinationName, exception);
+            log.error("[{}] Failed to terminated topic {}", clientAppId(), topicName, exception);
             throw new RestException(exception);
         }
     }
 
     protected void internalExpireMessages(String subName, int expireTimeInSeconds, boolean authoritative) {
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+        PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
         if (partitionMetadata.partitions > 0) {
-            // expire messages for each partition destination
+            // expire messages for each partition topic
             try {
                 for (int i = 0; i < partitionMetadata.partitions; i++) {
                     pulsar().getAdminClient().persistentTopics()
-                            .expireMessages(destinationName.getPartition(i).toString(), subName, expireTimeInSeconds);
+                            .expireMessages(topicName.getPartition(i).toString(), subName, expireTimeInSeconds);
                 }
             } catch (Exception e) {
                 throw new RestException(e);
             }
         } else {
             // validate ownership and redirect if current broker is not owner
-            validateAdminOperationOnDestination(authoritative);
-            if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
-                log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+            validateAdminOperationOnTopic(authoritative);
+            if (!(getTopicReference(topicName) instanceof PersistentTopic)) {
+                log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), topicName,
                         subName);
                 throw new RestException(Status.METHOD_NOT_ALLOWED,
                         "Expire messages on a non-persistent topic is not allowed");
             }
-            PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+            PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
             try {
                 if (subName.startsWith(topic.replicatorPrefix)) {
                     String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -1060,50 +1060,50 @@ public class PersistentTopicsBase extends AdminResource {
                     sub.expireMessages(expireTimeInSeconds);
                 }
                 log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds,
-                        destinationName, subName);
+                        topicName, subName);
             } catch (NullPointerException npe) {
                 throw new RestException(Status.NOT_FOUND, "Subscription not found");
             } catch (Exception exception) {
                 log.error("[{}] Failed to expire messages up to {} on {} with subscription {} {}", clientAppId(),
-                        expireTimeInSeconds, destinationName, subName, exception);
+                        expireTimeInSeconds, topicName, subName, exception);
                 throw new RestException(exception);
             }
         }
     }
 
     public static CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(PulsarService pulsar,
-            String clientAppId, AuthenticationDataSource authenticationData, DestinationName dn) {
+            String clientAppId, AuthenticationDataSource authenticationData, TopicName topicName) {
         CompletableFuture<PartitionedTopicMetadata> metadataFuture = new CompletableFuture<>();
         try {
             // (1) authorize client
             try {
-                checkAuthorization(pulsar, dn, clientAppId, authenticationData);
+                checkAuthorization(pulsar, topicName, clientAppId, authenticationData);
             } catch (RestException e) {
                 try {
-                    validateAdminAccessOnProperty(pulsar, clientAppId, dn.getProperty());
+                    validateAdminAccessOnProperty(pulsar, clientAppId, topicName.getProperty());
                 } catch (RestException authException) {
-                    log.warn("Failed to authorize {} on cluster {}", clientAppId, dn.toString());
+                    log.warn("Failed to authorize {} on cluster {}", clientAppId, topicName.toString());
                     throw new PulsarClientException(String.format("Authorization failed %s on topic %s with error %s",
-                            clientAppId, dn.toString(), authException.getMessage()));
+                            clientAppId, topicName.toString(), authException.getMessage()));
                 }
             } catch (Exception ex) {
                 // throw without wrapping to PulsarClientException that considers: unknown error marked as internal
                 // server error
                 log.warn("Failed to authorize {} on cluster {} with unexpected exception {}", clientAppId,
-                        dn.toString(), ex.getMessage(), ex);
+                        topicName.toString(), ex.getMessage(), ex);
                 throw ex;
             }
 
-            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getNamespace(), dn.getDomain().toString(),
-                    dn.getEncodedLocalName());
+            String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getNamespace(), topicName.getDomain().toString(),
+                    topicName.getEncodedLocalName());
 
             // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
             // serve/redirect request else fail partitioned-metadata-request so, client fails while creating
             // producer/consumer
-            checkLocalOrGetPeerReplicationCluster(pulsar, dn.getNamespaceObject())
+            checkLocalOrGetPeerReplicationCluster(pulsar, topicName.getNamespaceObject())
                     .thenCompose(res -> fetchPartitionedTopicMetadataAsync(pulsar, path)).thenAccept(metadata -> {
                         if (log.isDebugEnabled()) {
-                            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, dn,
+                            log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, topicName,
                                     metadata.partitions);
                         }
                         metadataFuture.complete(metadata);
@@ -1120,9 +1120,9 @@ public class PersistentTopicsBase extends AdminResource {
     /**
      * Get the Topic object reference from the Pulsar broker
      */
-    private Topic getTopicReference(DestinationName dn) {
+    private Topic getTopicReference(TopicName topicName) {
         try {
-            Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+            Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
             checkNotNull(topic);
             return topic;
         } catch (Exception e) {
@@ -1130,9 +1130,9 @@ public class PersistentTopicsBase extends AdminResource {
         }
     }
 
-    private Topic getOrCreateTopic(DestinationName dn) {
+    private Topic getOrCreateTopic(TopicName topicName) {
         try {
-            return pulsar().getBrokerService().getTopic(dn.toString()).get();
+            return pulsar().getBrokerService().getTopic(topicName.toString()).get();
         } catch (InterruptedException | ExecutionException e) {
            throw new RestException(e);
         }
@@ -1163,12 +1163,12 @@ public class PersistentTopicsBase extends AdminResource {
         }
     }
 
-    private CompletableFuture<Void> updatePartitionedTopic(DestinationName dn, int numPartitions) {
-        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(),
-                domain(), dn.getEncodedLocalName());
+    private CompletableFuture<Void> updatePartitionedTopic(TopicName topicName, int numPartitions) {
+        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getProperty(), topicName.getCluster(), topicName.getNamespacePortion(),
+                domain(), topicName.getEncodedLocalName());
 
         CompletableFuture<Void> updatePartition = new CompletableFuture<>();
-        createSubscriptions(dn, numPartitions).thenAccept(res -> {
+        createSubscriptions(topicName, numPartitions).thenAccept(res -> {
             try {
                 byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
                 globalZk().setData(path, data, -1, (rc, path1, ctx, stat) -> {
@@ -1193,14 +1193,14 @@ public class PersistentTopicsBase extends AdminResource {
     /**
      * It creates subscriptions for new partitions of existing partitioned-topics
      *
-     * @param dn
+     * @param topicName
      *            : topic-name: persistent://prop/cluster/ns/topic
      * @param numPartitions
      *            : number partitions for the topics
      */
-    private CompletableFuture<Void> createSubscriptions(DestinationName dn, int numPartitions) {
-        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(),
-                domain(), dn.getEncodedLocalName());
+    private CompletableFuture<Void> createSubscriptions(TopicName topicName, int numPartitions) {
+        String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getProperty(), topicName.getCluster(),
+                topicName.getNamespacePortion(), domain(), topicName.getEncodedLocalName());
         CompletableFuture<Void> result = new CompletableFuture<>();
         fetchPartitionedTopicMetadataAsync(pulsar(), path).thenAccept(partitionMetadata -> {
             if (partitionMetadata.partitions <= 1) {
@@ -1222,21 +1222,21 @@ public class PersistentTopicsBase extends AdminResource {
                 return;
             }
 
-            admin.persistentTopics().getStatsAsync(dn.getPartition(0).toString()).thenAccept(stats -> {
+            admin.persistentTopics().getStatsAsync(topicName.getPartition(0).toString()).thenAccept(stats -> {
                 stats.subscriptions.keySet().forEach(subscription -> {
                     List<CompletableFuture<Void>> subscriptionFutures = new ArrayList<>();
                     for (int i = partitionMetadata.partitions; i < numPartitions; i++) {
-                        final String topicName = dn.getPartition(i).toString();
+                        final String topicNamePartition = topicName.getPartition(i).toString();
 
-                        subscriptionFutures.add(admin.persistentTopics().createSubscriptionAsync(topicName,
+                        subscriptionFutures.add(admin.persistentTopics().createSubscriptionAsync(topicNamePartition,
                                 subscription, MessageId.latest));
                     }
 
                     FutureUtil.waitForAll(subscriptionFutures).thenRun(() -> {
-                        log.info("[{}] Successfully created new partitions {}", clientAppId(), dn);
+                        log.info("[{}] Successfully created new partitions {}", clientAppId(), topicName);
                         result.complete(null);
                     }).exceptionally(ex -> {
-                        log.warn("[{}] Failed to create subscriptions on new partitions for {}", clientAppId(), dn, ex);
+                        log.warn("[{}] Failed to create subscriptions on new partitions for {}", clientAppId(), topicName, ex);
                         result.completeExceptionally(ex);
                         return null;
                     });
@@ -1246,31 +1246,31 @@ public class PersistentTopicsBase extends AdminResource {
                     // The first partition doesn't exist, so there are currently to subscriptions to recreate
                     result.complete(null);
                 } else {
-                    log.warn("[{}] Failed to get list of subscriptions of {}", clientAppId(), dn.getPartition(0), ex);
+                    log.warn("[{}] Failed to get list of subscriptions of {}", clientAppId(), topicName.getPartition(0), ex);
                     result.completeExceptionally(ex);
                 }
                 return null;
             });
         }).exceptionally(ex -> {
-            log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), dn.toString());
+            log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), topicName.toString());
             result.completeExceptionally(ex);
             return null;
         });
         return result;
     }
 
-    protected void unloadTopic(DestinationName destination, boolean authoritative) {
+    protected void unloadTopic(TopicName topicName, boolean authoritative) {
         validateSuperUserAccess();
-        validateDestinationOwnership(destination, authoritative);
+        validateTopicOwnership(topicName, authoritative);
         try {
-            Topic topic = getTopicReference(destination);
+            Topic topic = getTopicReference(topicName);
             topic.close().get();
-            log.info("[{}] Successfully unloaded topic {}", clientAppId(), destination);
+            log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName);
         } catch (NullPointerException e) {
-            log.error("[{}] topic {} not found", clientAppId(), destination);
+            log.error("[{}] topic {} not found", clientAppId(), topicName);
             throw new RestException(Status.NOT_FOUND, "Topic does not exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to unload topic {}, {}", clientAppId(), destination, e.getCause().getMessage(), e);
+            log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, e.getCause().getMessage(), e);
             throw new RestException(e.getCause());
         }
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
index 6845e91..4cb6d5f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
@@ -161,7 +161,7 @@ public class PropertiesBase extends AdminResource {
 
     @DELETE
     @Path("/{property}")
-    @ApiOperation(value = "elete a property and all namespaces and destinations under it.")
+    @ApiOperation(value = "elete a property and all namespaces and topics under it.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Property does not exist"),
             @ApiResponse(code = 409, message = "The property still has active namespaces") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
index 3ae1d4b..ddd3154 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
@@ -69,6 +69,7 @@ import io.swagger.annotations.ApiResponses;
 @Produces(MediaType.APPLICATION_JSON)
 @Consumes(MediaType.APPLICATION_JSON)
 @Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
+@SuppressWarnings("deprecation")
 public class Namespaces extends NamespacesBase {
 
     @GET
@@ -112,10 +113,10 @@ public class Namespaces extends NamespacesBase {
 
     @GET
     @Path("/{property}/{cluster}/{namespace}/destinations")
-    @ApiOperation(hidden = true, value = "Get the list of all the destinations under a certain namespace.", response = String.class, responseContainer = "Set")
+    @ApiOperation(hidden = true, value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public List<String> getDestinations(@PathParam("property") String property,
+    public List<String> getTopics(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
         validateAdminAccessOnProperty(property);
         validateNamespaceName(property, cluster, namespace);
@@ -124,7 +125,7 @@ public class Namespaces extends NamespacesBase {
         getNamespacePolicies(namespaceName);
 
         try {
-            return pulsar().getNamespaceService().getListOfDestinations(namespaceName);
+            return pulsar().getNamespaceService().getListOfTopics(namespaceName);
         } catch (Exception e) {
             log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e);
             throw new RestException(e);
@@ -178,7 +179,7 @@ public class Namespaces extends NamespacesBase {
 
     @DELETE
     @Path("/{property}/{cluster}/{namespace}")
-    @ApiOperation(hidden = true, value = "Delete a namespace and all the destinations under it.")
+    @ApiOperation(hidden = true, value = "Delete a namespace and all the topics under it.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
             @ApiResponse(code = 409, message = "Namespace is not empty") })
@@ -191,7 +192,7 @@ public class Namespaces extends NamespacesBase {
 
     @DELETE
     @Path("/{property}/{cluster}/{namespace}/{bundle}")
-    @ApiOperation(hidden = true, value = "Delete a namespace bundle and all the destinations under it.")
+    @ApiOperation(hidden = true, value = "Delete a namespace bundle and all the topics under it.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
             @ApiResponse(code = 409, message = "Namespace bundle is not empty") })
@@ -460,7 +461,7 @@ public class Namespaces extends NamespacesBase {
     @PUT
     @Path("/{property}/{cluster}/{namespace}/unload")
     @ApiOperation(hidden = true, value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
-            + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+            + "removes all producers, consumers, and connections using this namespace, and close all topics (including"
             + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
             + "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
             + "result in non-persistent message loss and unexpected connection closure to the clients.")
@@ -533,7 +534,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/backlogQuota")
-    @ApiOperation(hidden = true, value = " Set a backlog quota for all the destinations on a namespace.")
+    @ApiOperation(hidden = true, value = " Set a backlog quota for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification"),
@@ -584,7 +585,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/persistence")
-    @ApiOperation(hidden = true, value = "Set the persistence configuration for all the destinations on a namespace.")
+    @ApiOperation(hidden = true, value = "Set the persistence configuration for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification"),
@@ -609,7 +610,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/clearBacklog")
-    @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace.")
+    @ApiOperation(hidden = true, value = "Clear backlog for all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -621,7 +622,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog")
-    @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace bundle.")
+    @ApiOperation(hidden = true, value = "Clear backlog for all topics on a namespace bundle.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBundleBacklog(@PathParam("property") String property,
@@ -634,7 +635,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}")
-    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace.")
+    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
@@ -647,7 +648,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}")
-    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+    @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all topics on a namespace bundle.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
@@ -660,7 +661,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}")
-    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace.")
+    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -672,7 +673,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}")
-    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace bundle.")
+    @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all topics on a namespace bundle.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -685,7 +686,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode")
-    @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+    @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
index c423f21..0f9ffe1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
@@ -37,7 +37,7 @@ import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.naming.Constants;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
@@ -63,96 +63,97 @@ import java.util.concurrent.ExecutionException;
 @Path("/non-persistent")
 @Produces(MediaType.APPLICATION_JSON)
 @Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic")
+@SuppressWarnings("deprecation")
 public class NonPersistentTopics extends PersistentTopics {
     private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        return getPartitionedTopicMetadata(destinationName, authoritative);
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        return getPartitionedTopicMetadata(topicName, authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/stats")
+    @Path("{property}/{cluster}/{namespace}/{topic}/stats")
     @ApiOperation(hidden = true, value = "Get the stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public NonPersistentTopicStats getStats(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        validateAdminOperationOnDestination(authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnTopic(authoritative);
+        Topic topic = getTopicReference(topicName);
         return ((NonPersistentTopic) topic).getStats();
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
+    @Path("{property}/{cluster}/{namespace}/{topic}/internalStats")
     @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        validateAdminOperationOnDestination(authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        validateAdminOperationOnTopic(authoritative);
+        Topic topic = getTopicReference(topicName);
         return topic.getInternalStats();
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic already exist") })
     public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        validateAdminAccessOnProperty(destinationName.getProperty());
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        validateAdminAccessOnProperty(topicName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
             String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
-                    destinationName.getEncodedLocalName());
+                    topicName.getEncodedLocalName());
             byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
             zkCreateOptimistic(path, data);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
         } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
             throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
             throw new RestException(e);
         }
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/unload")
     @ApiOperation(hidden = true, value = "Unload a topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        log.info("[{}] Unloading topic {}", clientAppId(), topicName);
 
-        if (destinationName.isGlobal()) {
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        unloadTopic(destinationName, authoritative);
+        unloadTopic(topicName, authoritative);
     }
 
     @GET
@@ -236,7 +237,7 @@ public class NonPersistentTopics extends PersistentTopics {
         try {
             final List<String> topicList = Lists.newArrayList();
             pulsar().getBrokerService().getTopics().forEach((name, topicFuture) -> {
-                DestinationName topicName = DestinationName.get(name);
+                TopicName topicName = TopicName.get(name);
                 if (nsBundle.includes(topicName)) {
                     topicList.add(name);
                 }
@@ -248,14 +249,14 @@ public class NonPersistentTopics extends PersistentTopics {
         }
     }
 
-    protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
-        validateAdminAccessOnProperty(fqdn.getProperty());
-        validateDestinationOwnership(fqdn, authoritative);
+    protected void validateAdminOperationOnTopic(TopicName topicName, boolean authoritative) {
+        validateAdminAccessOnProperty(topicName.getProperty());
+        validateTopicOwnership(topicName, authoritative);
     }
 
-    private Topic getTopicReference(DestinationName dn) {
+    private Topic getTopicReference(TopicName topicName) {
         try {
-            Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+            Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
             checkNotNull(topic);
             return topic;
         } catch (Exception e) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
index 624a3f3..8e10542 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
@@ -57,7 +57,9 @@ import io.swagger.annotations.ApiResponses;
 @Path("/persistent")
 @Produces(MediaType.APPLICATION_JSON)
 @Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
+@SuppressWarnings("deprecation")
 public class PersistentTopics extends PersistentTopicsBase {
+
     @GET
     @Path("/{property}/{cluster}/{namespace}")
     @ApiOperation(hidden = true, value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List")
@@ -81,55 +83,55 @@ public class PersistentTopics extends PersistentTopicsBase {
     }
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions")
-    @ApiOperation(hidden = true, value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
-            + "namespace level combined (union) with any eventual specific permission set on the destination.")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/permissions")
+    @ApiOperation(hidden = true, value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the"
+            + "namespace level combined (union) with any eventual specific permission set on the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist") })
-    public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
+    public Map<String, Set<AuthAction>> getPermissionsOnTopic(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        return internalGetPermissionsOnDestination();
+            @PathParam("topic") @Encoded String encodedTopic) {
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        return internalGetPermissionsOnTopic();
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/permissions/{role}")
     @ApiOperation(hidden = true, value = "Grant a new permission to a role on a single topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist"),
             @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void grantPermissionsOnDestination(@PathParam("property") String property,
+    public void grantPermissionsOnTopic(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("role") String role,
             Set<AuthAction> actions) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        internalGrantPermissionsOnDestination(role, actions);
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        internalGrantPermissionsOnTopic(role, actions);
     }
 
     @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
-    @ApiOperation(hidden = true, value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+    @Path("/{property}/{cluster}/{namespace}/{topic}/permissions/{role}")
+    @ApiOperation(hidden = true, value = "Revoke permissions on a topic.", notes = "Revoke permissions to a role on a single topic. If the permission was not set at the topic"
             + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
-    public void revokePermissionsOnDestination(@PathParam("property") String property,
+            @ApiResponse(code = 412, message = "Permissions are not set at the topic level") })
+    public void revokePermissionsOnTopic(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
-        internalRevokePermissionsOnDestination(role);
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("role") String role) {
+        validateTopicName(property, cluster, namespace, encodedTopic);
+        internalRevokePermissionsOnTopic(role);
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic already exist") })
     public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalCreatePartitionedTopic(numPartitions, authoritative);
     }
 
@@ -148,274 +150,274 @@ public class PersistentTopics extends PersistentTopicsBase {
      * @param numPartitions
      */
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic does not exist") })
     public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             int numPartitions) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalUpdatePartitionedTopic(numPartitions);
     }
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetPartitionedMetadata(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
     @ApiOperation(hidden = true, value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Partitioned topic does not exist") })
     public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalDeletePartitionedTopic(authoritative);
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/unload")
     @ApiOperation(hidden = true, value = "Unload a topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalUnloadTopic(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}")
     @ApiOperation(hidden = true, value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist"),
             @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
     public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalDeleteTopic(authoritative);
     }
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscriptions")
     @ApiOperation(hidden = true, value = "Get the list of persistent subscriptions for a given topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public List<String> getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetSubscriptions(authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/stats")
+    @Path("{property}/{cluster}/{namespace}/{topic}/stats")
     @ApiOperation(hidden = true, value = "Get the stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetStats(authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
+    @Path("{property}/{cluster}/{namespace}/{topic}/internalStats")
     @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetInternalStats(authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/internal-info")
+    @Path("{property}/{cluster}/{namespace}/{topic}/internal-info")
     @ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @Suspended AsyncResponse asyncResponse) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalGetManagedLedgerInfo(asyncResponse);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats")
+    @Path("{property}/{cluster}/{namespace}/{topic}/partitioned-stats")
     @ApiOperation(hidden = true, value = "Get the stats for the partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetPartitionedStats(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}")
     @ApiOperation(hidden = true, value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist"),
             @ApiResponse(code = 412, message = "Subscription has active consumers") })
     public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalDeleteSubscription(subName, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skip_all")
     @ApiOperation(hidden = true, value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalSkipAllMessages(subName, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}")
     @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName, @PathParam("numMessages") int numMessages,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalSkipMessages(subName, numMessages, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
     @ApiOperation(hidden = true, value = "Expire messages on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalExpireMessages(subName, expireTimeInSeconds, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}")
     @ApiOperation(hidden = true, value = "Expire messages on all subscriptions of topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/resetcursor/{timestamp}")
     @ApiOperation(hidden = true, value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
     public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName, @PathParam("timestamp") long timestamp,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalResetCursor(subName, timestamp, authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/resetcursor")
     @ApiOperation(hidden = true, value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
             @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
     public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         internalResetCursorOnPosition(subName, authoritative, messageId);
     }
 
     @PUT
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subscriptionName}")
     @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
             @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
     public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
-           @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
+           @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String topic,
            @PathParam("subscriptionName") String subscriptionName,
            @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
-        validateDestinationName(property, cluster, namespace, destination);
+        validateTopicName(property, cluster, namespace, topic);
         internalCreateSubscription(subscriptionName, messageId, authoritative);
     }
 
     @GET
-    @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}")
     @ApiOperation(hidden = true, value = "Peek nth message on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
     public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalPeekNthMessage(subName, messagePosition, authoritative);
     }
 
     @GET
-    @Path("{property}/{cluster}/{namespace}/{destination}/backlog")
+    @Path("{property}/{cluster}/{namespace}/{topic}/backlog")
     @ApiOperation(hidden = true, value = "Get estimated backlog for offline topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
             @PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalGetBacklog(authoritative);
     }
 
     @POST
-    @Path("/{property}/{cluster}/{namespace}/{destination}/terminate")
+    @Path("/{property}/{cluster}/{namespace}/{topic}/terminate")
     @ApiOperation(hidden = true, value = "Terminate a topic. A topic that is terminated will not accept any more "
             + "messages to be published and will let consumer to drain existing messages in backlog")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, cluster, namespace, encodedTopic);
+        validateTopicName(property, cluster, namespace, encodedTopic);
         return internalTerminate(authoritative);
     }
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index d00499a..873d761 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -73,7 +73,7 @@ public class Namespaces extends NamespacesBase {
     @ApiOperation(value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
-    public List<String> getDestinations(@PathParam("property") String property,
+    public List<String> getTopics(@PathParam("property") String property,
             @PathParam("namespace") String namespace) {
         validateAdminAccessOnProperty(property);
         validateNamespaceName(property, namespace);
@@ -82,7 +82,7 @@ public class Namespaces extends NamespacesBase {
         getNamespacePolicies(namespaceName);
 
         try {
-            return pulsar().getNamespaceService().getListOfDestinations(namespaceName);
+            return pulsar().getNamespaceService().getListOfTopics(namespaceName);
         } catch (Exception e) {
             log.error("Failed to get topics list for namespace {}", namespaceName, e);
             throw new RestException(e);
@@ -263,7 +263,7 @@ public class Namespaces extends NamespacesBase {
     @PUT
     @Path("/{property}/{namespace}/unload")
     @ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
-            + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+            + "removes all producers, consumers, and connections using this namespace, and close all topics (including"
             + "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
             + "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
             + "result in non-persistent message loss and unexpected connection closure to the clients.")
@@ -335,7 +335,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/backlogQuota")
-    @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.")
+    @ApiOperation(value = " Set a backlog quota for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification"),
@@ -384,7 +384,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/persistence")
-    @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.")
+    @ApiOperation(value = "Set the persistence configuration for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification"),
@@ -409,7 +409,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/clearBacklog")
-    @ApiOperation(value = "Clear backlog for all destinations on a namespace.")
+    @ApiOperation(value = "Clear backlog for all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("namespace") String namespace,
@@ -420,7 +420,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/{bundle}/clearBacklog")
-    @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.")
+    @ApiOperation(value = "Clear backlog for all topics on a namespace bundle.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBundleBacklog(@PathParam("property") String property,
@@ -432,7 +432,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/clearBacklog/{subscription}")
-    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.")
+    @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
@@ -444,7 +444,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/{bundle}/clearBacklog/{subscription}")
-    @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+    @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace bundle.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
@@ -457,7 +457,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/unsubscribe/{subscription}")
-    @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.")
+    @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -482,7 +482,7 @@ public class Namespaces extends NamespacesBase {
 
     @POST
     @Path("/{property}/{namespace}/subscriptionAuthMode")
-    @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+    @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist"),
             @ApiResponse(code = 409, message = "Concurrent modification") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
index 55bd9be..493030b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
@@ -35,7 +35,7 @@ import javax.ws.rs.core.Response.Status;
 import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
 import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
@@ -58,98 +58,98 @@ public class NonPersistentTopics extends PersistentTopics {
     private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
 
     @GET
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Get partitioned topic metadata.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        return getPartitionedTopicMetadata(destinationName, authoritative);
+        validateTopicName(property, namespace, encodedTopic);
+        return getPartitionedTopicMetadata(topicName, authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/stats")
+    @Path("{property}/{namespace}/{topic}/stats")
     @ApiOperation(value = "Get the stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public NonPersistentTopicStats getStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        validateAdminOperationOnDestination(destinationName, authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicName(property, namespace, encodedTopic);
+        validateAdminOperationOnTopic(topicName, authoritative);
+        Topic topic = getTopicReference(topicName);
         return ((NonPersistentTopic) topic).getStats();
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/internalStats")
+    @Path("{property}/{namespace}/{topic}/internalStats")
     @ApiOperation(value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        validateAdminOperationOnDestination(destinationName, authoritative);
-        Topic topic = getTopicReference(destinationName);
+        validateTopicName(property, namespace, encodedTopic);
+        validateAdminOperationOnTopic(topicName, authoritative);
+        Topic topic = getTopicReference(topicName);
         return topic.getInternalStats();
     }
 
     @PUT
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic already exist") })
     public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+            @PathParam("topic") @Encoded String encodedTopic, int numPartitions,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        validateAdminAccessOnProperty(destinationName.getProperty());
+        validateTopicName(property, namespace, encodedTopic);
+        validateAdminAccessOnProperty(topicName.getProperty());
         if (numPartitions <= 1) {
             throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
         }
         try {
             String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
-                    destinationName.getEncodedLocalName());
+                    topicName.getEncodedLocalName());
             byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
             zkCreateOptimistic(path, data);
             // we wait for the data to be synced in all quorums and the observers
             Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
-            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+            log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
         } catch (KeeperException.NodeExistsException e) {
-            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+            log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
             throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
         } catch (Exception e) {
-            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+            log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
             throw new RestException(e);
         }
     }
 
     @PUT
-    @Path("/{property}/{namespace}/{destination}/unload")
+    @Path("/{property}/{namespace}/{topic}/unload")
     @ApiOperation(value = "Unload a topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
-        log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
-        if (destinationName.isGlobal()) {
+        validateTopicName(property, namespace, encodedTopic);
+        log.info("[{}] Unloading topic {}", clientAppId(), topicName);
+        if (topicName.isGlobal()) {
             validateGlobalNamespaceOwnership(namespaceName);
         }
-        unloadTopic(destinationName, authoritative);
+        unloadTopic(topicName, authoritative);
     }
 
-    protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
-        validateAdminAccessOnProperty(fqdn.getProperty());
-        validateDestinationOwnership(fqdn, authoritative);
+    protected void validateAdminOperationOnTopic(TopicName topicName, boolean authoritative) {
+        validateAdminAccessOnProperty(topicName.getProperty());
+        validateTopicOwnership(topicName, authoritative);
     }
 
-    private Topic getTopicReference(DestinationName dn) {
+    private Topic getTopicReference(TopicName topicName) {
         try {
-            Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+            Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
             checkNotNull(topic);
             return topic;
         } catch (Exception e) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java
index b52a5ad..0f89bfd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java
@@ -81,53 +81,53 @@ public class PersistentTopics extends PersistentTopicsBase {
     }
 
     @GET
-    @Path("/{property}/{namespace}/{destination}/permissions")
-    @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
-            + "namespace level combined (union) with any eventual specific permission set on the destination.")
+    @Path("/{property}/{namespace}/{topic}/permissions")
+    @ApiOperation(value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the"
+            + "namespace level combined (union) with any eventual specific permission set on the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist") })
-    public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic) {
-        validateDestinationName(property, namespace, encodedTopic);
-        return internalGetPermissionsOnDestination();
+    public Map<String, Set<AuthAction>> getPermissionsOnTopic(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic) {
+        validateTopicName(property, namespace, encodedTopic);
+        return internalGetPermissionsOnTopic();
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/permissions/{role}")
+    @Path("/{property}/{namespace}/{topic}/permissions/{role}")
     @ApiOperation(value = "Grant a new permission to a role on a single topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist"),
             @ApiResponse(code = 409, message = "Concurrent modification") })
-    public void grantPermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+    public void grantPermissionsOnTopic(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("role") String role, Set<AuthAction> actions) {
-        validateDestinationName(property, namespace, encodedTopic);
-        internalGrantPermissionsOnDestination(role, actions);
+        validateTopicName(property, namespace, encodedTopic);
+        internalGrantPermissionsOnTopic(role, actions);
     }
 
     @DELETE
-    @Path("/{property}/{namespace}/{destination}/permissions/{role}")
-    @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+    @Path("/{property}/{namespace}/{topic}/permissions/{role}")
+    @ApiOperation(value = "Revoke permissions on a topic.", notes = "Revoke permissions to a role on a single topic. If the permission was not set at the topic"
             + "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace doesn't exist"),
-            @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
-    public void revokePermissionsOnDestination(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @ApiResponse(code = 412, message = "Permissions are not set at the topic level") })
+    public void revokePermissionsOnTopic(@PathParam("property") String property,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("role") String role) {
-        validateDestinationName(property, namespace, encodedTopic);
-        internalRevokePermissionsOnDestination(role);
+        validateTopicName(property, namespace, encodedTopic);
+        internalRevokePermissionsOnTopic(role);
     }
 
     @PUT
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic already exist") })
     public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+            @PathParam("topic") @Encoded String encodedTopic, int numPartitions,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalCreatePartitionedTopic(numPartitions, authoritative);
     }
 
@@ -143,254 +143,254 @@ public class PersistentTopics extends PersistentTopicsBase {
      * @param property
      * @param cluster
      * @param namespace
-     * @param destination
+     * @param topic
      * @param numPartitions
      */
     @POST
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 409, message = "Partitioned topic does not exist") })
     public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, int numPartitions) {
-        validateDestinationName(property, namespace, encodedTopic);
+            @PathParam("topic") @Encoded String encodedTopic, int numPartitions) {
+        validateTopicName(property, namespace, encodedTopic);
         internalUpdatePartitionedTopic(numPartitions);
     }
 
     @GET
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Get partitioned topic metadata.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
     public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetPartitionedMetadata(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{namespace}/{destination}/partitions")
+    @Path("/{property}/{namespace}/{topic}/partitions")
     @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Partitioned topic does not exist") })
     public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalDeletePartitionedTopic(authoritative);
     }
 
     @PUT
-    @Path("/{property}/{namespace}/{destination}/unload")
+    @Path("/{property}/{namespace}/{topic}/unload")
     @ApiOperation(value = "Unload a topic")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalUnloadTopic(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{namespace}/{destination}")
+    @Path("/{property}/{namespace}/{topic}")
     @ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist"),
             @ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
     public void deleteTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalDeleteTopic(authoritative);
     }
 
     @GET
-    @Path("/{property}/{namespace}/{destination}/subscriptions")
+    @Path("/{property}/{namespace}/{topic}/subscriptions")
     @ApiOperation(value = "Get the list of persistent subscriptions for a given topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public List<String> getSubscriptions(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetSubscriptions(authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/stats")
+    @Path("{property}/{namespace}/{topic}/stats")
     @ApiOperation(value = "Get the stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicStats getStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetStats(authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/internalStats")
+    @Path("{property}/{namespace}/{topic}/internalStats")
     @ApiOperation(value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetInternalStats(authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/internal-info")
+    @Path("{property}/{namespace}/{topic}/internal-info")
     @ApiOperation(value = "Get the internal stats for the topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) {
-        validateDestinationName(property, namespace, encodedTopic);
+            @PathParam("topic") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) {
+        validateTopicName(property, namespace, encodedTopic);
         internalGetManagedLedgerInfo(asyncResponse);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/partitioned-stats")
+    @Path("{property}/{namespace}/{topic}/partitioned-stats")
     @ApiOperation(value = "Get the stats for the partitioned topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetPartitionedStats(authoritative);
     }
 
     @DELETE
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}")
     @ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic does not exist"),
             @ApiResponse(code = 412, message = "Subscription has active consumers") })
     public void deleteSubscription(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalDeleteSubscription(subName, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip_all")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/skip_all")
     @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void skipAllMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalSkipAllMessages(subName, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}")
     @ApiOperation(value = "Skip messages on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void skipMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @PathParam("numMessages") int numMessages,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalSkipMessages(subName, numMessages, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
     @ApiOperation(value = "Expire messages on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void expireTopicMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalExpireMessages(subName, expireTimeInSeconds, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+    @Path("/{property}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}")
     @ApiOperation(value = "Expire messages on all subscriptions of topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic or subscription does not exist") })
     public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/resetcursor/{timestamp}")
     @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
     public void resetCursor(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @PathParam("timestamp") long timestamp,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalResetCursor(subName, timestamp, authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/resetcursor")
     @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
             @ApiResponse(code = 405, message = "Not supported for partitioned topics") })
     public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         internalResetCursorOnPosition(subName, authoritative, messageId);
     }
 
     @GET
-    @Path("/{property}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+    @Path("/{property}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}")
     @ApiOperation(value = "Peek nth message on a topic subscription.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
     public Response peekNthMessage(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+            @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
             @PathParam("messagePosition") int messagePosition,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalPeekNthMessage(subName, messagePosition, authoritative);
     }
 
     @GET
-    @Path("{property}/{namespace}/{destination}/backlog")
+    @Path("{property}/{namespace}/{topic}/backlog")
     @ApiOperation(value = "Get estimated backlog for offline topic.")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 404, message = "Namespace does not exist") })
     public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
-            @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalGetBacklog(authoritative);
     }
 
     @POST
-    @Path("/{property}/{namespace}/{destination}/terminate")
+    @Path("/{property}/{namespace}/{topic}/terminate")
     @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more "
             + "messages to be published and will let consumer to drain existing messages in backlog")
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
             @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
             @ApiResponse(code = 404, message = "Topic does not exist") })
     public MessageId terminate(@PathParam("property") String property, @PathParam("namespace") String namespace,
-            @PathParam("destination") @Encoded String encodedTopic,
+            @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
-        validateDestinationName(property, namespace, encodedTopic);
+        validateTopicName(property, namespace, encodedTopic);
         return internalTerminate(authoritative);
     }
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
index 6baf283..8e34353 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
@@ -39,7 +39,7 @@ public class SimpleLoadCalculatorImpl implements LoadCalculator {
     @Override
     public void recaliberateResourceUsagePerServiceUnit(LoadReport loadReport) {
         // Based on each load report, calculate the average resource required by each service request characteristics
-        // i.e. from the load report, we calculate that # of destinations and # of consumers are related to memory
+        // i.e. from the load report, we calculate that # of topics and # of consumers are related to memory
         // usage, # of msg/s is counted toward NIC inbw and outbw and CPU, # of connections are related to the # of
         // threads, etc.
         // For example, based on the the current usage number of CPU on the broker and the total # of msg/s in all
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
similarity index 85%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
index 300bd84..eed3681 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pulsar.broker.lookup;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
 import static org.apache.pulsar.common.api.Commands.newLookupResponse;
 
@@ -40,6 +39,7 @@ import javax.ws.rs.container.AsyncResponse;
 import javax.ws.rs.container.Suspended;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
 
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
@@ -49,14 +49,12 @@ import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType;
 import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
 import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.Codec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import javax.ws.rs.core.Response.Status;
 
 import io.netty.buffer.ByteBuf;
 import io.swagger.annotations.ApiResponse;
@@ -64,26 +62,26 @@ import io.swagger.annotations.ApiResponses;
 
 @Path("/v2/destination/")
 @NoSwaggerDocumentation
-public class DestinationLookup extends PulsarWebResource {
+public class TopicLookup extends PulsarWebResource {
 
     @GET
-    @Path("{destination-domain}/{property}/{cluster}/{namespace}/{dest}")
+    @Path("{topic-domain}/{property}/{cluster}/{namespace}/{topic}")
     @Produces(MediaType.APPLICATION_JSON)
-    public void lookupDestinationAsync(@PathParam("destination-domain") String destinationDomain,
+    public void lookupTopicAsync(@PathParam("topic-domain") String topicDomain,
             @PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("dest") @Encoded String dest,
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
             @QueryParam("authoritative") @DefaultValue("false") boolean authoritative,
             @Suspended AsyncResponse asyncResponse) {
-        dest = Codec.decode(dest);
-        DestinationDomain domain = null;
+        String topicName = Codec.decode(encodedTopic);
+        TopicDomain domain = null;
         try {
-            domain = DestinationDomain.getEnum(destinationDomain);
+            domain = TopicDomain.getEnum(topicDomain);
         } catch (IllegalArgumentException e) {
-            log.error("[{}] Invalid destination-domain {}", clientAppId(), destinationDomain, e);
+            log.error("[{}] Invalid topic-domain {}", clientAppId(), topicDomain, e);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
-                    "Unsupported destination domain " + destinationDomain);
+                    "Unsupported topic domain " + topicDomain);
         }
-        DestinationName topic = DestinationName.get(domain.value(), property, cluster, namespace, dest);
+        TopicName topic = TopicName.get(domain.value(), property, cluster, namespace, topicName);
 
         if (!pulsar().getBrokerService().getLookupRequestSemaphore().tryAcquire()) {
             log.warn("No broker was found available for topic {}", topic);
@@ -153,23 +151,23 @@ public class DestinationLookup extends PulsarWebResource {
     }
 
     @GET
-    @Path("{destination-domain}/{property}/{cluster}/{namespace}/{dest}/bundle")
+    @Path("{topic-domain}/{property}/{cluster}/{namespace}/{topic}/bundle")
     @Produces(MediaType.APPLICATION_JSON)
     @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
-            @ApiResponse(code = 405, message = "Invalid destination domain type") })
-    public String getNamespaceBundle(@PathParam("destination-domain") String destinationDomain,
+            @ApiResponse(code = 405, message = "Invalid topic domain type") })
+    public String getNamespaceBundle(@PathParam("topic-domain") String topicDomain,
             @PathParam("property") String property, @PathParam("cluster") String cluster,
-            @PathParam("namespace") String namespace, @PathParam("dest") @Encoded String dest) {
-        dest = Codec.decode(dest);
-        DestinationDomain domain = null;
+            @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String topicName) {
+        topicName = Codec.decode(topicName);
+        TopicDomain domain = null;
         try {
-            domain = DestinationDomain.getEnum(destinationDomain);
+            domain = TopicDomain.getEnum(topicDomain);
         } catch (IllegalArgumentException e) {
-            log.error("[{}] Invalid destination-domain {}", clientAppId(), destinationDomain, e);
+            log.error("[{}] Invalid topic-domain {}", clientAppId(), topicDomain, e);
             throw new RestException(Status.METHOD_NOT_ALLOWED,
-                    "Bundle lookup can not be done on destination domain " + destinationDomain);
+                    "Bundle lookup can not be done on topic domain " + topicDomain);
         }
-        DestinationName topic = DestinationName.get(domain.value(), property, cluster, namespace, dest);
+        TopicName topic = TopicName.get(domain.value(), property, cluster, namespace, topicName);
         validateSuperUserAccess();
         try {
             NamespaceBundle bundle = pulsar().getNamespaceService().getBundle(topic);
@@ -193,18 +191,18 @@ public class DestinationLookup extends PulsarWebResource {
      * success(connect) response to client.
      *
      * @param pulsarService
-     * @param fqdn
+     * @param topicName
      * @param authoritative
      * @param clientAppId
      * @param requestId
      * @return
      */
-    public static CompletableFuture<ByteBuf> lookupDestinationAsync(PulsarService pulsarService, DestinationName fqdn, boolean authoritative,
+    public static CompletableFuture<ByteBuf> lookupTopicAsync(PulsarService pulsarService, TopicName topicName, boolean authoritative,
             String clientAppId, AuthenticationDataSource authenticationData, long requestId) {
 
         final CompletableFuture<ByteBuf> validationFuture = new CompletableFuture<>();
         final CompletableFuture<ByteBuf> lookupfuture = new CompletableFuture<>();
-        final String cluster = fqdn.getCluster();
+        final String cluster = topicName.getCluster();
 
         // (1) validate cluster
         getClusterDataIfDifferentCluster(pulsarService, cluster, clientAppId).thenAccept(differentClusterData -> {
@@ -220,19 +218,19 @@ public class DestinationLookup extends PulsarWebResource {
             } else {
                 // (2) authorize client
                 try {
-                    checkAuthorization(pulsarService, fqdn, clientAppId, authenticationData);
+                    checkAuthorization(pulsarService, topicName, clientAppId, authenticationData);
                 } catch (RestException authException) {
-                    log.warn("Failed to authorized {} on cluster {}", clientAppId, fqdn.toString());
+                    log.warn("Failed to authorized {} on cluster {}", clientAppId, topicName.toString());
                     validationFuture.complete(
                             newLookupErrorResponse(ServerError.AuthorizationError, authException.getMessage(), requestId));
                     return;
                 } catch (Exception e) {
-                    log.warn("Unknown error while authorizing {} on cluster {}", clientAppId, fqdn.toString());
+                    log.warn("Unknown error while authorizing {} on cluster {}", clientAppId, topicName.toString());
                     validationFuture.completeExceptionally(e);
                     return;
                 }
                 // (3) validate global namespace
-                checkLocalOrGetPeerReplicationCluster(pulsarService, fqdn.getNamespaceObject())
+                checkLocalOrGetPeerReplicationCluster(pulsarService, topicName.getNamespaceObject())
                         .thenAccept(peerClusterData -> {
                             if (peerClusterData == null) {
                                 // (4) all validation passed: initiate lookup
@@ -261,16 +259,16 @@ public class DestinationLookup extends PulsarWebResource {
             if (validaitonFailureResponse != null) {
                 lookupfuture.complete(validaitonFailureResponse);
             } else {
-                pulsarService.getNamespaceService().getBrokerServiceUrlAsync(fqdn, authoritative)
+                pulsarService.getNamespaceService().getBrokerServiceUrlAsync(topicName, authoritative)
                         .thenAccept(lookupResult -> {
 
                             if (log.isDebugEnabled()) {
-                                log.debug("[{}] Lookup result {}", fqdn.toString(), lookupResult);
+                                log.debug("[{}] Lookup result {}", topicName.toString(), lookupResult);
                             }
 
                             if (!lookupResult.isPresent()) {
                                 lookupfuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady,
-                                        "No broker was available to own " + fqdn, requestId));
+                                        "No broker was available to own " + topicName, requestId));
                                 return;
                             }
 
@@ -287,10 +285,10 @@ public class DestinationLookup extends PulsarWebResource {
                             }
                         }).exceptionally(ex -> {
                             if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
-                                log.info("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+                                log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
                                         ex.getCause().getMessage());
                             } else {
-                                log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+                                log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
                                         ex.getMessage(), ex);
                             }
                             lookupfuture.complete(
@@ -301,10 +299,10 @@ public class DestinationLookup extends PulsarWebResource {
 
         }).exceptionally(ex -> {
             if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
-                log.info("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+                log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
                         ex.getCause().getMessage());
             } else {
-                log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+                log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
                         ex.getMessage(), ex);
             }
 
@@ -325,5 +323,5 @@ public class DestinationLookup extends PulsarWebResource {
         asyncResponse.resume(lookupData);
     }
 
-    private static final Logger log = LoggerFactory.getLogger(DestinationLookup.class);
+    private static final Logger log = LoggerFactory.getLogger(TopicLookup.class);
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
index 5319344..594b717 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
@@ -55,7 +55,7 @@ import org.apache.pulsar.broker.lookup.LookupResult;
 import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -132,19 +132,19 @@ public class NamespaceService {
         this.ownershipCache = new OwnershipCache(pulsar, bundleFactory);
     }
 
-    public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(DestinationName topic,
+    public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicName topic,
             boolean authoritative) {
         return getBundleAsync(topic)
                 .thenCompose(bundle -> findBrokerServiceUrl(bundle, authoritative, false /* read-only */));
     }
 
-    public CompletableFuture<NamespaceBundle> getBundleAsync(DestinationName topic) {
+    public CompletableFuture<NamespaceBundle> getBundleAsync(TopicName topic) {
         return bundleFactory.getBundlesAsync(topic.getNamespaceObject())
                 .thenApply(bundles -> bundles.findBundle(topic));
     }
 
-    public NamespaceBundle getBundle(DestinationName destination) throws Exception {
-        return bundleFactory.getBundles(destination.getNamespaceObject()).findBundle(destination);
+    public NamespaceBundle getBundle(TopicName topicName) throws Exception {
+        return bundleFactory.getBundles(topicName.getNamespaceObject()).findBundle(topicName);
     }
 
     public int getBundleCount(NamespaceName namespace) throws Exception {
@@ -162,10 +162,10 @@ public class NamespaceService {
      */
 	public Optional<URL> getWebServiceUrl(ServiceUnitId suName, boolean authoritative, boolean isRequestHttps, boolean readOnly)
 			throws Exception {
-        if (suName instanceof DestinationName) {
-            DestinationName name = (DestinationName) suName;
+        if (suName instanceof TopicName) {
+            TopicName name = (TopicName) suName;
             if (LOG.isDebugEnabled()) {
-                LOG.debug("Getting web service URL of destination: {} - auth: {}", name, authoritative);
+                LOG.debug("Getting web service URL of topic: {} - auth: {}", name, authoritative);
             }
             return this.internalGetWebServiceUrl(getBundle(name), authoritative, isRequestHttps, readOnly).get();
         }
@@ -248,7 +248,7 @@ public class NamespaceService {
             if (myUrl.equals(otherUrl)) {
                 if (nsFullBundle != null) {
                     // preload heartbeat namespace
-                    pulsar.loadNamespaceDestinations(nsFullBundle);
+                    pulsar.loadNamespaceTopics(nsFullBundle);
                 }
                 return true;
             }
@@ -372,8 +372,8 @@ public class NamespaceService {
                     } else {
                         // Found owner for the namespace bundle
 
-                        // Schedule the task to pre-load destinations
-                        pulsar.loadNamespaceDestinations(bundle);
+                        // Schedule the task to pre-load topics
+                        pulsar.loadNamespaceTopics(bundle);
 
                         lookupFuture.complete(Optional.of(new LookupResult(ownerInfo)));
                     }
@@ -667,8 +667,8 @@ public class NamespaceService {
     }
 
     public boolean isServiceUnitOwned(ServiceUnitId suName) throws Exception {
-        if (suName instanceof DestinationName) {
-            return isDestinationOwned((DestinationName) suName);
+        if (suName instanceof TopicName) {
+            return isTopicOwned((TopicName) suName);
         }
 
         if (suName instanceof NamespaceName) {
@@ -682,11 +682,11 @@ public class NamespaceService {
         throw new IllegalArgumentException("Invalid class of NamespaceBundle: " + suName.getClass().getName());
     }
 
-    public boolean isServiceUnitActive(DestinationName fqdn) {
+    public boolean isServiceUnitActive(TopicName topicName) {
         try {
-            return ownershipCache.getOwnedBundle(getBundle(fqdn)).isActive();
+            return ownershipCache.getOwnedBundle(getBundle(topicName)).isActive();
         } catch (Exception e) {
-            LOG.warn("Unable to find OwnedBundle for fqdn - [{}]", fqdn.toString());
+            LOG.warn("Unable to find OwnedBundle for topic - [{}]", topicName);
             return false;
         }
     }
@@ -695,12 +695,12 @@ public class NamespaceService {
         return ownershipCache.getOwnedBundle(getFullBundle(fqnn)) != null;
     }
 
-    private CompletableFuture<Boolean> isDestinationOwnedAsync(DestinationName topic) {
+    private CompletableFuture<Boolean> isTopicOwnedAsync(TopicName topic) {
         return getBundleAsync(topic).thenApply(bundle -> ownershipCache.isNamespaceBundleOwned(bundle));
     }
 
-    private boolean isDestinationOwned(DestinationName fqdn) throws Exception {
-        return ownershipCache.getOwnedBundle(getBundle(fqdn)) != null;
+    private boolean isTopicOwned(TopicName topicName) throws Exception {
+        return ownershipCache.getOwnedBundle(getBundle(topicName)) != null;
     }
 
     public void removeOwnedServiceUnit(NamespaceName nsName) throws Exception {
@@ -722,12 +722,12 @@ public class NamespaceService {
         return bundleFactory;
     }
 
-    public ServiceUnitId getServiceUnitId(DestinationName destinationName) throws Exception {
-        return getBundle(destinationName);
+    public ServiceUnitId getServiceUnitId(TopicName topicName) throws Exception {
+        return getBundle(topicName);
     }
 
-    public List<String> getListOfDestinations(NamespaceName namespaceName) throws Exception {
-        List<String> destinations = Lists.newArrayList();
+    public List<String> getListOfTopics(NamespaceName namespaceName) throws Exception {
+        List<String> topics = Lists.newArrayList();
 
         // For every topic there will be a managed ledger created.
         try {
@@ -736,15 +736,15 @@ public class NamespaceService {
                 LOG.debug("Getting children from managed-ledgers now: {}", path);
             }
 
-            for (String destination : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
-                destinations.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(destination)));
+            for (String topic : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
+                topics.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(topic)));
             }
         } catch (KeeperException.NoNodeException e) {
             // NoNode means there are no persistent topics for this namespace
         }
 
-        destinations.sort(null);
-        return destinations;
+        topics.sort(null);
+        return topics;
     }
 
     public Optional<NamespaceEphemeralData> getOwner(NamespaceBundle bundle) throws Exception {
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 4642a85..1919008 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
@@ -32,7 +32,7 @@ import org.apache.pulsar.client.api.ProducerBuilder;
 import org.apache.pulsar.client.impl.Backoff;
 import org.apache.pulsar.client.impl.ProducerImpl;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -217,28 +217,28 @@ public abstract class AbstractReplicator {
 
     /**
      * Replication can't be started on root-partitioned-topic to avoid producer startup conflict.
-     * 
+     *
      * <pre>
      * eg:
      * if topic : persistent://prop/cluster/ns/my-topic is a partitioned topic with 2 partitions then
      * broker explicitly creates replicator producer for: "my-topic-partition-1" and "my-topic-partition-2".
-     * 
-     * However, if broker tries to start producer with root topic "my-topic" then client-lib internally creates individual 
-     * producers for "my-topic-partition-1" and "my-topic-partition-2" which creates conflict with existing 
+     *
+     * However, if broker tries to start producer with root topic "my-topic" then client-lib internally creates individual
+     * producers for "my-topic-partition-1" and "my-topic-partition-2" which creates conflict with existing
      * replicator producers.
      * </pre>
-     * 
+     *
      * Therefore, replicator can't be started on root-partition topic which can internally create multiple partitioned
      * producers.
-     * 
+     *
      * @param topicName
      * @param brokerService
      */
-    private void validatePartitionedTopic(String topicName, BrokerService brokerService) throws NamingException {
-        DestinationName destination = DestinationName.get(topicName);
+    private void validatePartitionedTopic(String topic, BrokerService brokerService) throws NamingException {
+        TopicName topicName = TopicName.get(topic);
         String partitionedTopicPath = path(AdminResource.PARTITIONED_TOPIC_PATH_ZNODE,
-                destination.getNamespace().toString(), destination.getDomain().toString(),
-                destination.getEncodedLocalName());
+                topicName.getNamespace().toString(), topicName.getDomain().toString(),
+                topicName.getEncodedLocalName());
         boolean isPartitionedTopic = false;
         try {
             isPartitionedTopic = brokerService.pulsar().getConfigurationCache().policiesCache()
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
index 556a1f8..e7e38dd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
@@ -27,7 +27,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
@@ -82,8 +82,8 @@ public class BacklogQuotaManager {
      *            Topic on which backlog has been exceeded
      */
     public void handleExceededBacklogQuota(PersistentTopic persistentTopic) {
-        DestinationName destination = DestinationName.get(persistentTopic.getName());
-        String namespace = destination.getNamespace();
+        TopicName topicName = TopicName.get(persistentTopic.getName());
+        String namespace = topicName.getNamespace();
         String policyPath = AdminResource.path(POLICIES, namespace);
 
         BacklogQuota quota = getBacklogQuota(namespace, policyPath);
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 0c8fa43..20b1ce4 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
@@ -71,9 +71,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceExcept
 import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException;
 import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
 import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
-import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter;
 import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
-import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
 import org.apache.pulsar.broker.web.PulsarWebResource;
@@ -85,13 +83,12 @@ import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.common.configuration.FieldContext;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ClusterData;
-import org.apache.pulsar.common.policies.data.DispatchRate;
 import org.apache.pulsar.common.policies.data.PersistencePolicies;
 import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicStats;
@@ -148,7 +145,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     private final EventLoopGroup workerGroup;
     private final OrderedSafeExecutor topicOrderedExecutor;
     // offline topic backlog cache
-    private final ConcurrentOpenHashMap<DestinationName, PersistentOfflineTopicStats> offlineTopicStatCache;
+    private final ConcurrentOpenHashMap<TopicName, PersistentOfflineTopicStats> offlineTopicStatCache;
     private static final ConcurrentOpenHashMap<String, ConfigField> dynamicConfigurationMap = prepareDynamicConfigurationMap();
     private final ConcurrentOpenHashMap<String, Consumer> configRegisteredListeners;
 
@@ -428,7 +425,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                     return topicFuture;
                 }
             }
-            final boolean isPersistentTopic = DestinationName.get(topic).getDomain().equals(DestinationDomain.persistent);
+            final boolean isPersistentTopic = TopicName.get(topic).getDomain().equals(TopicDomain.persistent);
             return topics.computeIfAbsent(topic, (topicName) -> {
                 return isPersistentTopic ? this.createPersistentTopic(topicName)
                         : createNonPersistentTopic(topicName);
@@ -466,7 +463,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             log.info("Created topic {}", nonPersistentTopic);
             long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs;
             pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs);
-            addTopicToStatsMaps(DestinationName.get(topic), nonPersistentTopic);
+            addTopicToStatsMaps(TopicName.get(topic), nonPersistentTopic);
             topicFuture.complete(nonPersistentTopic);
         });
         replicationFuture.exceptionally((ex) -> {
@@ -568,8 +565,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     private void createPersistentTopic(final String topic, CompletableFuture<Topic> topicFuture) {
 
         final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime());
-        DestinationName destinationName = DestinationName.get(topic);
-        if (!pulsar.getNamespaceService().isServiceUnitActive(destinationName)) {
+        TopicName topicName = TopicName.get(topic);
+        if (!pulsar.getNamespaceService().isServiceUnitActive(topicName)) {
             // namespace is being unloaded
             String msg = String.format("Namespace is being unloaded, cannot add topic %s", topic);
             log.warn(msg);
@@ -578,9 +575,9 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             return;
         }
 
-        getManagedLedgerConfig(destinationName).thenAccept(managedLedgerConfig -> {
+        getManagedLedgerConfig(topicName).thenAccept(managedLedgerConfig -> {
             // Once we have the configuration, we can proceed with the async open operation
-            managedLedgerFactory.asyncOpen(destinationName.getPersistenceNamingEncoding(), managedLedgerConfig,
+            managedLedgerFactory.asyncOpen(topicName.getPersistenceNamingEncoding(), managedLedgerConfig,
                     new OpenLedgerCallback() {
                         @Override
                         public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
@@ -597,7 +594,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                                     long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime())
                                             - topicCreateTimeMs;
                                     pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs);
-                                    addTopicToStatsMaps(destinationName, persistentTopic);
+                                    addTopicToStatsMaps(topicName, persistentTopic);
                                     topicFuture.complete(persistentTopic);
                                 }).exceptionally((ex) -> {
                                     log.warn(
@@ -635,14 +632,14 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         });
     }
 
-    public CompletableFuture<ManagedLedgerConfig> getManagedLedgerConfig(DestinationName topicName) {
+    public CompletableFuture<ManagedLedgerConfig> getManagedLedgerConfig(TopicName topicName) {
         CompletableFuture<ManagedLedgerConfig> future = new CompletableFuture<>();
         // Execute in background thread, since getting the policies might block if the z-node wasn't already cached
         pulsar.getOrderedExecutor().submitOrdered(topicName, safeRun(() -> {
             NamespaceName namespace = topicName.getNamespaceObject();
             ServiceConfiguration serviceConfig = pulsar.getConfiguration();
 
-            // Get persistence policy for this destination
+            // Get persistence policy for this topic
             Optional<Policies> policies = Optional.empty();
             try {
                 policies = pulsar
@@ -698,7 +695,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         return future;
     }
 
-    private void addTopicToStatsMaps(DestinationName topicName, Topic topic) {
+    private void addTopicToStatsMaps(TopicName topicName, Topic topic) {
         try {
             NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(topicName);
 
@@ -726,7 +723,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             if (!isEmpty(topics)) {
                 // add topic under new split bundles which already updated into NamespaceBundleFactory.bundleCache
                 topics.stream().forEach(t -> {
-                    addTopicToStatsMaps(DestinationName.get(t.getName()), t);
+                    addTopicToStatsMaps(TopicName.get(t.getName()), t);
                 });
                 // remove old bundle from the map
                 synchronized (multiLayerTopicsMap) {
@@ -739,15 +736,15 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         }
     }
 
-    public PersistentOfflineTopicStats getOfflineTopicStat(DestinationName topicName) {
+    public PersistentOfflineTopicStats getOfflineTopicStat(TopicName topicName) {
         return offlineTopicStatCache.get(topicName);
     }
 
-    public void cacheOfflineTopicStats(DestinationName topicName, PersistentOfflineTopicStats offlineTopicStats) {
+    public void cacheOfflineTopicStats(TopicName topicName, PersistentOfflineTopicStats offlineTopicStats) {
         offlineTopicStatCache.put(topicName, offlineTopicStats);
     }
 
-    public void invalidateOfflineTopicStatCache(DestinationName topicName) {
+    public void invalidateOfflineTopicStatCache(TopicName topicName) {
         PersistentOfflineTopicStats removed = offlineTopicStatCache.remove(topicName);
         if (removed != null) {
             log.info("Removed cached offline topic stat for {} ", topicName.getPersistenceNamingEncoding());
@@ -773,8 +770,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         pulsarStats.getDimensionMetrics(consumer);
     }
 
-    public List<Metrics> getDestinationMetrics() {
-        return pulsarStats.getDestinationMetrics();
+    public List<Metrics> getTopicMetrics() {
+        return pulsarStats.getTopicMetrics();
     }
 
     public Map<String, NamespaceBundleStats> getBundleStats() {
@@ -823,8 +820,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
      * @return determine if quota enforcement needs to be done for topic
      */
     public boolean isBacklogExceeded(PersistentTopic topic) {
-        DestinationName destination = DestinationName.get(topic.getName());
-        long backlogQuotaLimitInBytes = getBacklogQuotaManager().getBacklogQuotaLimit(destination.getNamespace());
+        TopicName topicName = TopicName.get(topic.getName());
+        long backlogQuotaLimitInBytes = getBacklogQuotaManager().getBacklogQuotaLimit(topicName.getNamespace());
         if (log.isDebugEnabled()) {
             log.debug("[{}] - backlog quota limit = [{}]", topic.getName(), backlogQuotaLimitInBytes);
         }
@@ -863,18 +860,18 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     }
 
     void checkTopicNsOwnership(final String topic) throws RuntimeException {
-        DestinationName destination = DestinationName.get(topic);
+        TopicName topicName = TopicName.get(topic);
         boolean ownedByThisInstance;
         try {
-            ownedByThisInstance = pulsar.getNamespaceService().isServiceUnitOwned(destination);
+            ownedByThisInstance = pulsar.getNamespaceService().isServiceUnitOwned(topicName);
         } catch (Exception e) {
-            log.debug(String.format("Failed to check the ownership of the destination: %s", destination), e);
+            log.debug(String.format("Failed to check the ownership of the topic: %s", topicName), e);
             throw new RuntimeException(new ServerMetadataException(e));
         }
 
         if (!ownedByThisInstance) {
             String msg = String.format("Namespace not served by this instance. Please redo the lookup. "
-                    + "Request is denied: namespace=%s", destination.getNamespace());
+                    + "Request is denied: namespace=%s", topicName.getNamespace());
             log.warn(msg);
             throw new RuntimeException(new ServiceUnitNotReadyException(msg));
         }
@@ -890,7 +887,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         CompletableFuture<Integer> result = new CompletableFuture<Integer>();
         List<CompletableFuture<Void>> closeFutures = Lists.newArrayList();
         topics.forEach((name, topicFuture) -> {
-            DestinationName topicName = DestinationName.get(name);
+            TopicName topicName = TopicName.get(name);
             if (serviceUnit.includes(topicName)) {
                 // Topic needs to be unloaded
                 log.info("[{}] Unloading topic", topicName);
@@ -911,12 +908,12 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
 
     public void removeTopicFromCache(String topic) {
         try {
-            DestinationName destination = DestinationName.get(topic);
-            NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(destination);
+            TopicName topicName = TopicName.get(topic);
+            NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(topicName);
             checkArgument(namespaceBundle instanceof NamespaceBundle);
 
             String bundleName = namespaceBundle.toString();
-            String namespaceName = destination.getNamespaceObject().toString();
+            String namespaceName = topicName.getNamespaceObject().toString();
 
             synchronized (multiLayerTopicsMap) {
                 ConcurrentOpenHashMap<String, ConcurrentOpenHashMap<String, Topic>> namespaceMap = multiLayerTopicsMap
@@ -962,7 +959,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         log.info("{} updating with {}", path, data);
 
         topics.forEach((name, topicFuture) -> {
-            if (namespace.includes(DestinationName.get(name))) {
+            if (namespace.includes(TopicName.get(name))) {
                 // If the topic is already created, immediately apply the updated policies, otherwise once the topic is
                 // created it'll apply the policies update
                 topicFuture.thenAccept(topic -> {
@@ -1121,7 +1118,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             });
         });
     }
-    
+
     /**
      * Allows a listener to listen on update of {@link ServiceConfiguration} change, so listener can take appropriate
      * action if any specific config-field value has been changed.
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
index 0bb30cf..dc2e54e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
@@ -44,7 +44,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
 import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.util.DateFormatter;
 import org.slf4j.Logger;
@@ -481,19 +481,18 @@ public class Consumer {
     }
 
     public void checkPermissions() {
-        DestinationName destination = DestinationName.get(subscription.getDestination());
+        TopicName topicName = TopicName.get(subscription.getTopicName());
         if (cnx.getBrokerService().getAuthorizationService() != null) {
             try {
-                if (cnx.getBrokerService().getAuthorizationService().canConsume(destination, appId, authenticationData,
+                if (cnx.getBrokerService().getAuthorizationService().canConsume(topicName, appId, authenticationData,
                         subscription.getName())) {
                     return;
                 }
             } catch (Exception e) {
-                log.warn("[{}] Get unexpected error while autorizing [{}]  {}", appId, subscription.getDestination(),
+                log.warn("[{}] Get unexpected error while autorizing [{}]  {}", appId, subscription.getTopicName(),
                         e.getMessage(), e);
             }
-            log.info("[{}] is not allowed to consume from Destination" + " [{}] anymore", appId,
-                    subscription.getDestination());
+            log.info("[{}] is not allowed to consume from topic [{}] anymore", appId, subscription.getTopicName());
             disconnect();
         }
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
index c0de7a6..ffe3e1c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.common.api.Commands;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
 import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
 import org.apache.pulsar.common.policies.data.PublisherStats;
 import org.apache.pulsar.common.util.DateFormatter;
@@ -468,10 +468,10 @@ public class Producer {
     }
 
     public void checkPermissions() {
-        DestinationName destination = DestinationName.get(topic.getName());
+        TopicName topicName = TopicName.get(topic.getName());
         if (cnx.getBrokerService().getAuthorizationService() != null) {
             try {
-                if (cnx.getBrokerService().getAuthorizationService().canProduce(destination, appId,
+                if (cnx.getBrokerService().getAuthorizationService().canProduce(topicName, appId,
                         authenticationData)) {
                     return;
                 }
@@ -479,14 +479,14 @@ public class Producer {
                 log.warn("[{}] Get unexpected error while autorizing [{}]  {}", appId, topic.getName(), e.getMessage(),
                         e);
             }
-            log.info("[{}] is not allowed to produce from destination [{}] anymore", appId, topic.getName());
+            log.info("[{}] is not allowed to produce on topic [{}] anymore", appId, topic.getName());
             disconnect();
         }
     }
 
     public void checkEncryption() {
         if (topic.isEncryptionRequired() && !isEncrypted) {
-            log.info("[{}] [{}] Unencrypted producer is not allowed to produce from destination [{}] anymore",
+            log.info("[{}] [{}] Unencrypted producer is not allowed to produce on topic [{}] anymore",
                     producerId, producerName, topic.getName());
             disconnect();
         }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
index 835cabc..c04c7a8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
@@ -144,7 +144,7 @@ public class PulsarStats implements Closeable {
                         });
                         // end persistent topics section
                         topicStatsStream.endObject();
-                        
+
                         if(!tempNonPersistentTopics.isEmpty()) {
                          // start non-persistent topic
                             topicStatsStream.startObject("non-persistent");
@@ -157,9 +157,9 @@ public class PulsarStats implements Closeable {
                                 }
                             });
                             // end non-persistent topics section
-                            topicStatsStream.endObject();    
+                            topicStatsStream.endObject();
                         }
-                        
+
                         // end namespace-bundle section
                         topicStatsStream.endObject();
                     });
@@ -182,7 +182,7 @@ public class PulsarStats implements Closeable {
             // json end
             topicStatsStream.endObject();
         } catch (Exception e) {
-            log.error("Unable to update destination stats", e);
+            log.error("Unable to update topic stats", e);
         }
 
         // swap metricsCollection and tempMetricsCollection
@@ -214,7 +214,7 @@ public class PulsarStats implements Closeable {
         }
     }
 
-    public List<Metrics> getDestinationMetrics() {
+    public List<Metrics> getTopicMetrics() {
         return metricsCollection;
     }
 
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 4a2ecf8..935cb1f 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
@@ -21,7 +21,7 @@ package org.apache.pulsar.broker.service;
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
 import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.getPartitionedTopicMetadata;
-import static org.apache.pulsar.broker.lookup.DestinationLookup.lookupDestinationAsync;
+import static org.apache.pulsar.broker.lookup.TopicLookup.lookupTopicAsync;
 import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
 import static org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion.v5;
 
@@ -71,7 +71,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
 import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
 import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.Metadata;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
@@ -213,7 +213,7 @@ public class ServerCnx extends PulsarHandler {
             log.debug("[{}] Received Lookup from {} for {}", lookup.getTopic(), remoteAddress, requestId);
         }
 
-        DestinationName topicName = validateTopicName(lookup.getTopic(), requestId, lookup);
+        TopicName topicName = validateTopicName(lookup.getTopic(), requestId, lookup);
         if (topicName == null) {
             return;
         }
@@ -253,7 +253,7 @@ public class ServerCnx extends PulsarHandler {
             String finalOriginalPrincipal = originalPrincipal;
             isProxyAuthorizedFuture.thenApply(isProxyAuthorized -> {
                 if (isProxyAuthorized) {
-                    lookupDestinationAsync(getBrokerService().pulsar(), topicName, authoritative,
+                    lookupTopicAsync(getBrokerService().pulsar(), topicName, authoritative,
                             finalOriginalPrincipal != null ? finalOriginalPrincipal : authRole, authenticationData,
                             requestId).handle((lookupResponse, ex) -> {
                                 if (ex == null) {
@@ -299,7 +299,7 @@ public class ServerCnx extends PulsarHandler {
                     remoteAddress, requestId);
         }
 
-        DestinationName topicName = validateTopicName(partitionMetadata.getTopic(), requestId, partitionMetadata);
+        TopicName topicName = validateTopicName(partitionMetadata.getTopic(), requestId, partitionMetadata);
         if (topicName == null) {
             return;
         }
@@ -528,7 +528,7 @@ public class ServerCnx extends PulsarHandler {
         checkArgument(state == State.Connected);
         final long requestId = subscribe.getRequestId();
         final long consumerId = subscribe.getConsumerId();
-        DestinationName topicName = validateTopicName(subscribe.getTopic(), requestId, subscribe);
+        TopicName topicName = validateTopicName(subscribe.getTopic(), requestId, subscribe);
         if (topicName == null) {
             return;
         }
@@ -704,7 +704,7 @@ public class ServerCnx extends PulsarHandler {
         final boolean isEncrypted = cmdProducer.getEncrypted();
         final Map<String, String> metadata = CommandUtils.metadataFromCommand(cmdProducer);
 
-        DestinationName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer);
+        TopicName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer);
         if (topicName == null) {
             return;
         }
@@ -1125,7 +1125,7 @@ public class ServerCnx extends PulsarHandler {
 
             Topic topic = consumer.getSubscription().getTopic();
             Position position = topic.getLastMessageId();
-            int partitionIndex = DestinationName.getPartitionIndex(topic.getName());
+            int partitionIndex = TopicName.getPartitionIndex(topic.getName());
             if (log.isDebugEnabled()) {
                 log.debug("[{}] [{}][{}] Get LastMessageId {} partitionIndex {}", remoteAddress,
                     topic.getName(), consumer.getSubscription().getName(), position, partitionIndex);
@@ -1259,9 +1259,9 @@ public class ServerCnx extends PulsarHandler {
         }
     }
 
-    private DestinationName validateTopicName(String topic, long requestId, GeneratedMessageLite requestCommand) {
+    private TopicName validateTopicName(String topic, long requestId, GeneratedMessageLite requestCommand) {
         try {
-            return DestinationName.get(topic);
+            return TopicName.get(topic);
         } catch (Throwable t) {
             if (log.isDebugEnabled()) {
                 log.debug("[{}] Failed to parse topic name '{}'", remoteAddress, topic, t);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
index f50229a..65d613d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
@@ -42,7 +42,7 @@ public interface Subscription {
 
     void acknowledgeMessage(PositionImpl position, AckType ackType, Map<String,Long> properties);
 
-    String getDestination();
+    String getTopicName();
 
     Dispatcher getDispatcher();
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
index be5374d..0b0f6d3 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
@@ -36,7 +36,7 @@ import org.apache.pulsar.broker.service.Subscription;
 import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats;
 import org.apache.pulsar.utils.CopyOnWriteArrayList;
@@ -95,7 +95,7 @@ public class NonPersistentSubscription implements Subscription {
                 }
                 break;
             case Failover:
-                int partitionIndex = DestinationName.getPartitionIndex(topicName);
+                int partitionIndex = TopicName.getPartitionIndex(topicName);
                 if (partitionIndex < 0) {
                     // For non partition topics, assume index 0 to pick a predictable consumer
                     partitionIndex = 0;
@@ -149,7 +149,7 @@ public class NonPersistentSubscription implements Subscription {
     }
 
     @Override
-    public String getDestination() {
+    public String getTopicName() {
         return this.topicName;
     }
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
index 5941768..a03ac6c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
@@ -62,7 +62,7 @@ import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
 import org.apache.pulsar.broker.stats.NamespaceStats;
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
@@ -177,7 +177,7 @@ public class NonPersistentTopic implements Topic {
 
         try {
             Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
-                    .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+                    .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
                     .orElseThrow(() -> new KeeperException.NoNodeException());
             isEncryptionRequired = policies.encryption_required;
         } catch (Exception e) {
@@ -500,7 +500,7 @@ public class NonPersistentTopic implements Topic {
 
     @Override
     public CompletableFuture<Void> checkReplication() {
-        DestinationName name = DestinationName.get(topic);
+        TopicName name = TopicName.get(topic);
         if (!name.isGlobal()) {
             return CompletableFuture.completedFuture(null);
         }
@@ -667,7 +667,7 @@ public class NonPersistentTopic implements Topic {
         return topic;
     }
 
-    public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream destStatsStream,
+    public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream topicStatsStream,
             ClusterReplicationMetrics replStats, String namespace) {
 
         TopicStats topicStats = threadLocalTopicStats.get();
@@ -677,7 +677,7 @@ public class NonPersistentTopic implements Topic {
 
         nsStats.producerCount += producers.size();
         bundleStats.producerCount += producers.size();
-        destStatsStream.startObject(topic);
+        topicStatsStream.startObject(topic);
 
         producers.forEach(producer -> {
             producer.updateRates();
@@ -692,18 +692,18 @@ public class NonPersistentTopic implements Topic {
         });
 
         // Creating publishers object for backward compatibility
-        destStatsStream.startList("publishers");
-        destStatsStream.endList();
+        topicStatsStream.startList("publishers");
+        topicStatsStream.endList();
 
         // Start replicator stats
-        destStatsStream.startObject("replication");
+        topicStatsStream.startObject("replication");
         nsStats.replicatorCount += topicStats.remotePublishersStats.size();
 
         // Close replication
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
 
         // Start subscription stats
-        destStatsStream.startObject("subscriptions");
+        topicStatsStream.startObject("subscriptions");
         nsStats.subsCount += subscriptions.size();
 
         subscriptions.forEach((subscriptionName, subscription) -> {
@@ -713,12 +713,12 @@ public class NonPersistentTopic implements Topic {
 
             // Start subscription name & consumers
             try {
-                destStatsStream.startObject(subscriptionName);
+                topicStatsStream.startObject(subscriptionName);
                 Object[] consumers = subscription.getConsumers().array();
                 nsStats.consumerCount += consumers.length;
                 bundleStats.consumerCount += consumers.length;
 
-                destStatsStream.startList("consumers");
+                topicStatsStream.startList("consumers");
 
                 subscription.getDispatcher().getMesssageDropRate().calculateRate();
                 for (Object consumerObj : consumers) {
@@ -731,43 +731,43 @@ public class NonPersistentTopic implements Topic {
                     subMsgRateRedeliver += consumerStats.msgRateRedeliver;
 
                     // Populate consumer specific stats here
-                    destStatsStream.startObject();
-                    destStatsStream.writePair("address", consumerStats.address);
-                    destStatsStream.writePair("consumerName", consumerStats.consumerName);
-                    destStatsStream.writePair("availablePermits", consumerStats.availablePermits);
-                    destStatsStream.writePair("connectedSince", consumerStats.connectedSince);
-                    destStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
-                    destStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
-                    destStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
+                    topicStatsStream.startObject();
+                    topicStatsStream.writePair("address", consumerStats.address);
+                    topicStatsStream.writePair("consumerName", consumerStats.consumerName);
+                    topicStatsStream.writePair("availablePermits", consumerStats.availablePermits);
+                    topicStatsStream.writePair("connectedSince", consumerStats.connectedSince);
+                    topicStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
+                    topicStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
+                    topicStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
 
                     if (SubType.Shared.equals(subscription.getType())) {
-                        destStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
-                        destStatsStream.writePair("blockedConsumerOnUnackedMsgs",
+                        topicStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
+                        topicStatsStream.writePair("blockedConsumerOnUnackedMsgs",
                                 consumerStats.blockedConsumerOnUnackedMsgs);
                     }
                     if (consumerStats.clientVersion != null) {
-                        destStatsStream.writePair("clientVersion", consumerStats.clientVersion);
+                        topicStatsStream.writePair("clientVersion", consumerStats.clientVersion);
                     }
-                    destStatsStream.endObject();
+                    topicStatsStream.endObject();
                 }
 
                 // Close Consumer stats
-                destStatsStream.endList();
+                topicStatsStream.endList();
 
                 // Populate subscription specific stats here
-                destStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
-                destStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
-                destStatsStream.writePair("msgRateOut", subMsgRateOut);
-                destStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
-                destStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
-                destStatsStream.writePair("type", subscription.getTypeString());
+                topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
+                topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
+                topicStatsStream.writePair("msgRateOut", subMsgRateOut);
+                topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
+                topicStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
+                topicStatsStream.writePair("type", subscription.getTypeString());
                 if (subscription.getDispatcher() != null) {
-                    destStatsStream.writePair("msgDropRate",
+                    topicStatsStream.writePair("msgDropRate",
                             subscription.getDispatcher().getMesssageDropRate().getRate());
                 }
 
                 // Close consumers
-                destStatsStream.endObject();
+                topicStatsStream.endObject();
 
                 topicStats.aggMsgRateOut += subMsgRateOut;
                 topicStats.aggMsgThroughputOut += subMsgThroughputOut;
@@ -779,17 +779,17 @@ public class NonPersistentTopic implements Topic {
         });
 
         // Close subscription
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
 
         // Remaining dest stats.
         topicStats.averageMsgSize = topicStats.aggMsgRateIn == 0.0 ? 0.0
                 : (topicStats.aggMsgThroughputIn / topicStats.aggMsgRateIn);
-        destStatsStream.writePair("producerCount", producers.size());
-        destStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
-        destStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
-        destStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
-        destStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
-        destStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
+        topicStatsStream.writePair("producerCount", producers.size());
+        topicStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
+        topicStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
+        topicStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
+        topicStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
+        topicStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
 
         nsStats.msgRateIn += topicStats.aggMsgRateIn;
         nsStats.msgRateOut += topicStats.aggMsgRateOut;
@@ -802,7 +802,7 @@ public class NonPersistentTopic implements Topic {
         bundleStats.msgThroughputOut += topicStats.aggMsgThroughputOut;
 
         // Close topic object
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
     }
 
     public NonPersistentTopicStats getStats() {
@@ -867,7 +867,7 @@ public class NonPersistentTopic implements Topic {
     }
 
     public boolean isActive() {
-        if (DestinationName.get(topic).isGlobal()) {
+        if (TopicName.get(topic).isGlobal()) {
             // No local consumers and no local producers
             return !subscriptions.isEmpty() || hasLocalProducers();
         }
@@ -881,7 +881,7 @@ public class NonPersistentTopic implements Topic {
         } else {
             if (System.nanoTime() - lastActive > TimeUnit.SECONDS.toNanos(gcIntervalInSeconds)) {
 
-                if (DestinationName.get(topic).isGlobal()) {
+                if (TopicName.get(topic).isGlobal()) {
                     // For global namespace, close repl producers first.
                     // Once all repl producers are closed, we can delete the topic,
                     // provided no remote producers connected to the broker.
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
index dae7c4f..5bbd79a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
@@ -50,7 +50,7 @@ import org.apache.pulsar.broker.service.Subscription;
 import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.policies.data.SubscriptionStats;
 import org.apache.pulsar.compaction.CompactedTopic;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
index 882c56a..55dda83 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
@@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
 import org.apache.pulsar.broker.service.BrokerService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.DispatchRate;
 import org.apache.pulsar.common.policies.data.Policies;
@@ -114,7 +114,7 @@ public class DispatchRateLimiter {
      */
     private void registerLocalPoliciesListener() {
         brokerService.pulsar().getConfigurationCache().policiesCache().registerListener((path, data, stat) -> {
-            final NamespaceName namespace = DestinationName.get(this.topicName).getNamespaceObject();
+            final NamespaceName namespace = TopicName.get(this.topicName).getNamespaceObject();
             final String cluster = brokerService.pulsar().getConfiguration().getClusterName();
             final String policiesPath = path(POLICIES, namespace.toString());
             if (policiesPath.equals(path)) {
@@ -141,7 +141,7 @@ public class DispatchRateLimiter {
      * @return
      */
     public DispatchRate getPoliciesDispatchRate() {
-        final NamespaceName namespace = DestinationName.get(this.topicName).getNamespaceObject();
+        final NamespaceName namespace = TopicName.get(this.topicName).getNamespaceObject();
         final String cluster = brokerService.pulsar().getConfiguration().getClusterName();
         final String path = path(POLICIES, namespace.toString());
         Optional<Policies> policies = Optional.empty();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java
index 6333f3a..e7e50b8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java
@@ -42,7 +42,7 @@ import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.service.Topic.PublishContext;
 import org.apache.pulsar.common.api.Commands;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -365,7 +365,7 @@ public class MessageDeduplication {
     }
 
     private CompletableFuture<Boolean> isDeduplicationEnabled() {
-        DestinationName name = DestinationName.get(topic.getName());
+        TopicName name = TopicName.get(topic.getName());
 
         return pulsar.getConfigurationCache().policiesCache()
                 .getAsync(AdminResource.path(POLICIES, name.getNamespace())).thenApply(policies -> {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java
index e742f14..1f0ea31 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.Subscription;
 import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.policies.data.SubscriptionStats;
 import org.apache.pulsar.utils.CopyOnWriteArrayList;
@@ -119,7 +119,7 @@ public class PersistentSubscription implements Subscription {
                 }
                 break;
             case Failover:
-                int partitionIndex = DestinationName.getPartitionIndex(topicName);
+                int partitionIndex = TopicName.getPartitionIndex(topicName);
                 if (partitionIndex < 0) {
                     // For non partition topics, assume index 0 to pick a predictable consumer
                     partitionIndex = 0;
@@ -234,7 +234,7 @@ public class PersistentSubscription implements Subscription {
     }
 
     @Override
-    public String getDestination() {
+    public String getTopicName() {
         return this.topicName;
     }
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 88813ea..6e6b5d8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -81,7 +81,7 @@ import org.apache.pulsar.client.impl.BatchMessageIdImpl;
 import org.apache.pulsar.client.impl.MessageIdImpl;
 import org.apache.pulsar.client.impl.MessageImpl;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.ConsumerStats;
 import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
@@ -235,7 +235,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
 
         try {
             Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
-                    .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+                    .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
                     .orElseThrow(() -> new KeeperException.NoNodeException());
             isEncryptionRequired = policies.encryption_required;
         } catch (Exception e) {
@@ -381,7 +381,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         // read repl-cluster from policies to avoid restart of replicator which are in process of disconnect and close
         try {
             Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
-                    .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+                    .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
                     .orElseThrow(() -> new KeeperException.NoNodeException());
             if (policies.replication_clusters != null) {
                 Set<String> configuredClusters = Sets.newTreeSet(policies.replication_clusters);
@@ -791,7 +791,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
     }
 
     private CompletableFuture<Void> checkPersistencePolicies() {
-        DestinationName topicName = DestinationName.get(topic);
+        TopicName topicName = TopicName.get(topic);
         CompletableFuture<Void> future = new CompletableFuture<>();
         brokerService.getManagedLedgerConfig(topicName).thenAccept(config -> {
             // update managed-ledger config and managed-cursor.markDeleteRate
@@ -807,7 +807,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
 
     @Override
     public CompletableFuture<Void> checkReplication() {
-        DestinationName name = DestinationName.get(topic);
+        TopicName name = TopicName.get(topic);
         if (!name.isGlobal()) {
             return CompletableFuture.completedFuture(null);
         }
@@ -868,7 +868,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
 
     @Override
     public void checkMessageExpiry() {
-        DestinationName name = DestinationName.get(topic);
+        TopicName name = TopicName.get(topic);
         Policies policies;
         try {
             policies = brokerService.pulsar().getConfigurationCache().policiesCache()
@@ -1021,7 +1021,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         return ledger;
     }
 
-    public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream destStatsStream,
+    public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream topicStatsStream,
             ClusterReplicationMetrics replStats, String namespace) {
 
         TopicStats topicStats = threadLocalTopicStats.get();
@@ -1031,7 +1031,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
 
         nsStats.producerCount += producers.size();
         bundleStats.producerCount += producers.size();
-        destStatsStream.startObject(topic);
+        topicStatsStream.startObject(topic);
 
         producers.forEach(producer -> {
             producer.updateRates();
@@ -1046,11 +1046,11 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         });
 
         // Creating publishers object for backward compatibility
-        destStatsStream.startList("publishers");
-        destStatsStream.endList();
+        topicStatsStream.startList("publishers");
+        topicStatsStream.endList();
 
         // Start replicator stats
-        destStatsStream.startObject("replication");
+        topicStatsStream.startObject("replication");
         nsStats.replicatorCount += topicStats.remotePublishersStats.size();
         replicators.forEach((cluster, replicator) -> {
             // Update replicator cursor state
@@ -1072,20 +1072,20 @@ public class PersistentTopic implements Topic, AddEntryCallback {
             topicStats.aggMsgThroughputOut += rStat.msgThroughputOut;
 
             // Populate replicator specific stats here
-            destStatsStream.startObject(cluster);
-            destStatsStream.writePair("connected", rStat.connected);
-            destStatsStream.writePair("msgRateExpired", rStat.msgRateExpired);
-            destStatsStream.writePair("msgRateIn", rStat.msgRateIn);
-            destStatsStream.writePair("msgRateOut", rStat.msgRateOut);
-            destStatsStream.writePair("msgThroughputIn", rStat.msgThroughputIn);
-            destStatsStream.writePair("msgThroughputOut", rStat.msgThroughputOut);
-            destStatsStream.writePair("replicationBacklog", rStat.replicationBacklog);
-            destStatsStream.writePair("replicationDelayInSeconds", rStat.replicationDelayInSeconds);
-            destStatsStream.writePair("inboundConnection", rStat.inboundConnection);
-            destStatsStream.writePair("inboundConnectedSince", rStat.inboundConnectedSince);
-            destStatsStream.writePair("outboundConnection", rStat.outboundConnection);
-            destStatsStream.writePair("outboundConnectedSince", rStat.outboundConnectedSince);
-            destStatsStream.endObject();
+            topicStatsStream.startObject(cluster);
+            topicStatsStream.writePair("connected", rStat.connected);
+            topicStatsStream.writePair("msgRateExpired", rStat.msgRateExpired);
+            topicStatsStream.writePair("msgRateIn", rStat.msgRateIn);
+            topicStatsStream.writePair("msgRateOut", rStat.msgRateOut);
+            topicStatsStream.writePair("msgThroughputIn", rStat.msgThroughputIn);
+            topicStatsStream.writePair("msgThroughputOut", rStat.msgThroughputOut);
+            topicStatsStream.writePair("replicationBacklog", rStat.replicationBacklog);
+            topicStatsStream.writePair("replicationDelayInSeconds", rStat.replicationDelayInSeconds);
+            topicStatsStream.writePair("inboundConnection", rStat.inboundConnection);
+            topicStatsStream.writePair("inboundConnectedSince", rStat.inboundConnectedSince);
+            topicStatsStream.writePair("outboundConnection", rStat.outboundConnection);
+            topicStatsStream.writePair("outboundConnectedSince", rStat.outboundConnectedSince);
+            topicStatsStream.endObject();
 
             nsStats.msgReplBacklog += rStat.replicationBacklog;
 
@@ -1108,10 +1108,10 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         });
 
         // Close replication
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
 
         // Start subscription stats
-        destStatsStream.startObject("subscriptions");
+        topicStatsStream.startObject("subscriptions");
         nsStats.subsCount += subscriptions.size();
 
         subscriptions.forEach((subscriptionName, subscription) -> {
@@ -1121,12 +1121,12 @@ public class PersistentTopic implements Topic, AddEntryCallback {
 
             // Start subscription name & consumers
             try {
-                destStatsStream.startObject(subscriptionName);
+                topicStatsStream.startObject(subscriptionName);
                 Object[] consumers = subscription.getConsumers().array();
                 nsStats.consumerCount += consumers.length;
                 bundleStats.consumerCount += consumers.length;
 
-                destStatsStream.startList("consumers");
+                topicStatsStream.startList("consumers");
 
                 for (Object consumerObj : consumers) {
                     Consumer consumer = (Consumer) consumerObj;
@@ -1138,48 +1138,48 @@ public class PersistentTopic implements Topic, AddEntryCallback {
                     subMsgRateRedeliver += consumerStats.msgRateRedeliver;
 
                     // Populate consumer specific stats here
-                    destStatsStream.startObject();
-                    destStatsStream.writePair("address", consumerStats.address);
-                    destStatsStream.writePair("consumerName", consumerStats.consumerName);
-                    destStatsStream.writePair("availablePermits", consumerStats.availablePermits);
-                    destStatsStream.writePair("connectedSince", consumerStats.connectedSince);
-                    destStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
-                    destStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
-                    destStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
+                    topicStatsStream.startObject();
+                    topicStatsStream.writePair("address", consumerStats.address);
+                    topicStatsStream.writePair("consumerName", consumerStats.consumerName);
+                    topicStatsStream.writePair("availablePermits", consumerStats.availablePermits);
+                    topicStatsStream.writePair("connectedSince", consumerStats.connectedSince);
+                    topicStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
+                    topicStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
+                    topicStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
 
                     if (SubType.Shared.equals(subscription.getType())) {
-                        destStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
-                        destStatsStream.writePair("blockedConsumerOnUnackedMsgs",
+                        topicStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
+                        topicStatsStream.writePair("blockedConsumerOnUnackedMsgs",
                                 consumerStats.blockedConsumerOnUnackedMsgs);
                     }
                     if (consumerStats.clientVersion != null) {
-                        destStatsStream.writePair("clientVersion", consumerStats.clientVersion);
+                        topicStatsStream.writePair("clientVersion", consumerStats.clientVersion);
                     }
-                    destStatsStream.endObject();
+                    topicStatsStream.endObject();
                 }
 
                 // Close Consumer stats
-                destStatsStream.endList();
+                topicStatsStream.endList();
 
                 // Populate subscription specific stats here
-                destStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
-                destStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
-                destStatsStream.writePair("msgRateOut", subMsgRateOut);
-                destStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
-                destStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
-                destStatsStream.writePair("numberOfEntriesSinceFirstNotAckedMessage", subscription.getNumberOfEntriesSinceFirstNotAckedMessage());
-                destStatsStream.writePair("totalNonContiguousDeletedMessagesRange", subscription.getTotalNonContiguousDeletedMessagesRange());
-                destStatsStream.writePair("type", subscription.getTypeString());
+                topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
+                topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
+                topicStatsStream.writePair("msgRateOut", subMsgRateOut);
+                topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
+                topicStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
+                topicStatsStream.writePair("numberOfEntriesSinceFirstNotAckedMessage", subscription.getNumberOfEntriesSinceFirstNotAckedMessage());
+                topicStatsStream.writePair("totalNonContiguousDeletedMessagesRange", subscription.getTotalNonContiguousDeletedMessagesRange());
+                topicStatsStream.writePair("type", subscription.getTypeString());
                 if (SubType.Shared.equals(subscription.getType())) {
                     if(subscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) {
                         PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers)subscription.getDispatcher();
-                        destStatsStream.writePair("blockedSubscriptionOnUnackedMsgs",  dispatcher.isBlockedDispatcherOnUnackedMsgs());
-                        destStatsStream.writePair("unackedMessages", dispatcher.getTotalUnackedMessages());
+                        topicStatsStream.writePair("blockedSubscriptionOnUnackedMsgs",  dispatcher.isBlockedDispatcherOnUnackedMsgs());
+                        topicStatsStream.writePair("unackedMessages", dispatcher.getTotalUnackedMessages());
                     }
                 }
 
                 // Close consumers
-                destStatsStream.endObject();
+                topicStatsStream.endObject();
 
                 topicStats.aggMsgRateOut += subMsgRateOut;
                 topicStats.aggMsgThroughputOut += subMsgThroughputOut;
@@ -1191,19 +1191,19 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         });
 
         // Close subscription
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
 
         // Remaining dest stats.
         topicStats.averageMsgSize = topicStats.aggMsgRateIn == 0.0 ? 0.0
                 : (topicStats.aggMsgThroughputIn / topicStats.aggMsgRateIn);
-        destStatsStream.writePair("producerCount", producers.size());
-        destStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
-        destStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
-        destStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
-        destStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
-        destStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
-        destStatsStream.writePair("storageSize", ledger.getEstimatedBacklogSize());
-        destStatsStream.writePair("pendingAddEntriesCount", ((ManagedLedgerImpl) ledger).getPendingAddEntriesCount());
+        topicStatsStream.writePair("producerCount", producers.size());
+        topicStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
+        topicStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
+        topicStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
+        topicStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
+        topicStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
+        topicStatsStream.writePair("storageSize", ledger.getEstimatedBacklogSize());
+        topicStatsStream.writePair("pendingAddEntriesCount", ((ManagedLedgerImpl) ledger).getPendingAddEntriesCount());
 
         nsStats.msgRateIn += topicStats.aggMsgRateIn;
         nsStats.msgRateOut += topicStats.aggMsgRateOut;
@@ -1218,7 +1218,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         bundleStats.cacheSize += ((ManagedLedgerImpl) ledger).getCacheSize();
 
         // Close topic object
-        destStatsStream.endObject();
+        topicStatsStream.endObject();
     }
 
     public PersistentTopicStats getStats() {
@@ -1328,7 +1328,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
     }
 
     public boolean isActive() {
-        if (DestinationName.get(topic).isGlobal()) {
+        if (TopicName.get(topic).isGlobal()) {
             // No local consumers and no local producers
             return !subscriptions.isEmpty() || hasLocalProducers();
         }
@@ -1348,7 +1348,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
         } else {
             CompletableFuture<Void> replCloseFuture = new CompletableFuture<>();
 
-            if (DestinationName.get(topic).isGlobal()) {
+            if (TopicName.get(topic).isGlobal()) {
                 // For global namespace, close repl producers first.
                 // Once all repl producers are closed, we can delete the topic,
                 // provided no remote producers connected to the broker.
@@ -1402,7 +1402,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
      * marked as inactive.
      */
     private boolean shouldTopicBeRetained() {
-        DestinationName name = DestinationName.get(topic);
+        TopicName name = TopicName.get(topic);
         try {
             Optional<Policies> policies = brokerService.pulsar().getConfigurationCache().policiesCache()
                     .get(AdminResource.path(POLICIES, name.getNamespace()));
@@ -1448,8 +1448,8 @@ public class PersistentTopic implements Topic, AddEntryCallback {
      */
     @Override
     public BacklogQuota getBacklogQuota() {
-        DestinationName destination = DestinationName.get(this.getName());
-        String namespace = destination.getNamespace();
+        TopicName topicName = TopicName.get(this.getName());
+        String namespace = topicName.getNamespace();
         String policyPath = AdminResource.path(POLICIES, namespace);
 
         BacklogQuota backlogQuota = brokerService.getBacklogQuotaManager().getBacklogQuota(namespace, policyPath);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
index 1b67ea0..c2ffc53 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 
 import com.google.common.collect.Maps;
 
@@ -32,7 +32,7 @@ import com.google.common.collect.Maps;
 public class BookieClientStatsGenerator {
     private final PulsarService pulsar;
 
-    // map<namespace, map<destination, bookieOpsStats>>
+    // map<namespace, map<topic, bookieOpsStats>>
     private Map<String, Map<String, PendingBookieOpsStats>> nsBookieClientStatsMap;
 
     public BookieClientStatsGenerator(PulsarService pulsar) {
@@ -49,8 +49,8 @@ public class BookieClientStatsGenerator {
             pulsar.getBrokerService().getTopics().forEach((name, topicFuture) -> {
                 PersistentTopic persistentTopic = (PersistentTopic) topicFuture.getNow(null);
                 if (persistentTopic != null) {
-                    DestinationName destinationName = DestinationName.get(persistentTopic.getName());
-                    put(destinationName, persistentTopic.getManagedLedger().getStats().getPendingBookieOpsStats());
+                    TopicName topicName = TopicName.get(persistentTopic.getName());
+                    put(topicName, persistentTopic.getManagedLedger().getStats().getPendingBookieOpsStats());
                 }
             });
         }
@@ -58,14 +58,14 @@ public class BookieClientStatsGenerator {
         return nsBookieClientStatsMap;
     }
 
-    private void put(DestinationName destinationName, PendingBookieOpsStats bookieOpsStats) {
-        String namespace = destinationName.getNamespace();
+    private void put(TopicName topicName, PendingBookieOpsStats bookieOpsStats) {
+        String namespace = topicName.getNamespace();
         if (!nsBookieClientStatsMap.containsKey(namespace)) {
             Map<String, PendingBookieOpsStats> destBookieClientStatsMap = Maps.newTreeMap();
-            destBookieClientStatsMap.put(destinationName.toString(), bookieOpsStats);
+            destBookieClientStatsMap.put(topicName.toString(), bookieOpsStats);
             nsBookieClientStatsMap.put(namespace, destBookieClientStatsMap);
         } else {
-            nsBookieClientStatsMap.get(namespace).put(destinationName.toString(), bookieOpsStats);
+            nsBookieClientStatsMap.get(namespace).put(topicName.toString(), bookieOpsStats);
         }
 
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
index 62ac7ce..9472a7d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
@@ -52,7 +52,7 @@ public class MetricsGenerator {
         metricsCollection.addAll(jvmMetrics.generate());
         metricsCollection.addAll(new ManagedLedgerCacheMetrics(pulsar).generate());
         metricsCollection.addAll(new ManagedLedgerMetrics(pulsar).generate());
-        metricsCollection.addAll(pulsar.getBrokerService().getDestinationMetrics());
+        metricsCollection.addAll(pulsar.getBrokerService().getTopicMetrics());
         metricsCollection.addAll(pulsar.getLoadManager().get().getLoadBalancingMetrics());
 
         return metricsCollection;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
index 3716c75..f9c73e7 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
@@ -211,7 +211,7 @@ abstract class AbstractMetrics {
         Double val = map.getOrDefault(mkey, 0.0);
         map.put(mkey, val + value);
     }
-    
+
     protected void populateMaxMap(Map<String, Long> map, String mkey, long value) {
         Long existingValue = map.get(mkey);
         if (existingValue == null || value > existingValue) {
@@ -220,9 +220,9 @@ abstract class AbstractMetrics {
     }
 
     /**
-     * Helper to manage populating destination map
+     * Helper to manage populating topics map
      *
-     * @param destStatsByDimensionMap
+     * @param ledgersByDimensionMap
      * @param dimensionKey
      * @param destStats
      */
@@ -237,14 +237,14 @@ abstract class AbstractMetrics {
         }
     }
 
-    protected void populateDimensionMap(Map<Metrics, List<PersistentTopicStats>> destStatsByDimensionMap,
+    protected void populateDimensionMap(Map<Metrics, List<PersistentTopicStats>> topicsStatsByDimensionMap,
             Metrics metrics, PersistentTopicStats destStats) {
-        if (!destStatsByDimensionMap.containsKey(metrics)) {
+        if (!topicsStatsByDimensionMap.containsKey(metrics)) {
             // create new list
-            destStatsByDimensionMap.put(metrics, Lists.newArrayList(destStats));
+            topicsStatsByDimensionMap.put(metrics, Lists.newArrayList(destStats));
         } else {
             // add to collection
-            destStatsByDimensionMap.get(metrics).add(destStats);
+            topicsStatsByDimensionMap.get(metrics).add(destStats);
         }
 
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
index ea5f322..281a795 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
@@ -36,7 +36,7 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
     private Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap;
     // temp map to prepare aggregation metrics
     private Map<String, Double> tempAggregatedMetricsMap;
-    
+
     public ManagedLedgerMetrics(PulsarService pulsar) {
         super(pulsar);
         this.metricsCollection = Lists.newArrayList();
@@ -54,14 +54,14 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
 
     /**
      * Aggregation by namespace (not thread-safe)
-     * 
+     *
      * @param ledgersByDimension
      * @return
      */
     private List<Metrics> aggregate(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimension) {
 
         metricsCollection.clear();
-        
+
         for (Entry<Metrics, List<ManagedLedgerImpl>> e : ledgersByDimension.entrySet()) {
             Metrics metrics = e.getKey();
             List<ManagedLedgerImpl> ledgers = e.getValue();
@@ -122,17 +122,17 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
     }
 
     /**
-     * Build a map of dimensions key to list of destination stats (not thread-safe)
+     * Build a map of dimensions key to list of topic stats (not thread-safe)
      * <p>
-     * 
+     *
      * @return
      */
     private Map<Metrics, List<ManagedLedgerImpl>> groupLedgersByDimension() {
 
         ledgersByDimensionMap.clear();
-        
-        // get the current destinations statistics from StatsBrokerFilter
-        // Map : destination-name->dest-stat
+
+        // get the current topics statistics from StatsBrokerFilter
+        // Map : topic-name->dest-stat
 
         for (Entry<String, ManagedLedgerImpl> e : getManagedLedgers().entrySet()) {
 
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 60031fc..bae1019 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
@@ -128,7 +128,7 @@ public abstract class PulsarWebResource {
     public AuthenticationDataHttps clientAuthData() {
         return (AuthenticationDataHttps) httpRequest.getAttribute(AuthenticationFilter.AuthenticatedDataAttributeName);
     }
-    
+
     public boolean isRequestHttps() {
     	return "https".equalsIgnoreCase(httpRequest.getScheme());
     }
@@ -180,7 +180,7 @@ public abstract class PulsarWebResource {
         if (pulsar.getConfiguration().isAuthenticationEnabled() && pulsar.getConfiguration().isAuthorizationEnabled()) {
             log.debug("check admin access on property: {} - Authenticated: {} -- role: {}", property,
                     (isClientAuthenticated(clientAppId)), clientAppId);
-            
+
             PropertyAdmin propertyAdmin;
 
             try {
@@ -242,7 +242,7 @@ public abstract class PulsarWebResource {
         try {
             ClusterData differentClusterData = getClusterDataIfDifferentCluster(pulsar(), cluster, clientAppId()).get();
             if (differentClusterData != null) {
-                URI redirect = getRedirectionUrl(differentClusterData); 
+                URI redirect = getRedirectionUrl(differentClusterData);
                 // redirect to the cluster requested
                 if (log.isDebugEnabled()) {
                     log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, cluster);
@@ -478,19 +478,19 @@ public abstract class PulsarWebResource {
      * @param cluster
      * @param namespace
      */
-    protected void validateDestinationOwnership(DestinationName fqdn, boolean authoritative) {
+    protected void validateTopicOwnership(TopicName topicName, boolean authoritative) {
         NamespaceService nsService = pulsar().getNamespaceService();
 
         try {
             // per function name, this is trying to acquire the whole namespace ownership
-            Optional<URL> webUrl = nsService.getWebServiceUrl(fqdn, authoritative, isRequestHttps(), false);
+            Optional<URL> webUrl = nsService.getWebServiceUrl(topicName, authoritative, isRequestHttps(), false);
             // Ensure we get a url
             if (webUrl == null || !webUrl.isPresent()) {
                 log.info("Unable to get web service url");
-                throw new RestException(Status.PRECONDITION_FAILED, "Failed to find ownership for destination:" + fqdn);
+                throw new RestException(Status.PRECONDITION_FAILED, "Failed to find ownership for topic:" + topicName);
             }
 
-            if (!nsService.isServiceUnitOwned(fqdn)) {
+            if (!nsService.isServiceUnitOwned(topicName)) {
                 boolean newAuthoritative = isLeaderBroker(pulsar());
                 // Replace the host and port of the current request and redirect
                 URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(webUrl.get().getHost())
@@ -501,15 +501,15 @@ public abstract class PulsarWebResource {
             }
         } catch (IllegalArgumentException iae) {
             // namespace format is not valid
-            log.debug(String.format("Failed to find owner for destination:%s", fqdn), iae);
-            throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for destination " + fqdn);
+            log.debug(String.format("Failed to find owner for topic :%s", topicName), iae);
+            throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for topic " + topicName);
         } catch (IllegalStateException ise) {
-            log.debug(String.format("Failed to find owner for destination:%s", fqdn), ise);
-            throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for destination " + fqdn);
+            log.debug(String.format("Failed to find owner for topic:%s", topicName), ise);
+            throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for topic " + topicName);
         } catch (WebApplicationException wae) {
             throw wae;
         } catch (Exception oe) {
-            log.debug(String.format("Failed to find owner for destination:%s", fqdn), oe);
+            log.debug(String.format("Failed to find owner for topic:%s", topicName), oe);
             throw new RestException(oe);
         }
     }
@@ -636,19 +636,19 @@ public abstract class PulsarWebResource {
         return null;
     }
 
-    protected void checkConnect(DestinationName destination) throws RestException, Exception {
-        checkAuthorization(pulsar(), destination, clientAppId(), clientAuthData());
+    protected void checkConnect(TopicName topicName) throws RestException, Exception {
+        checkAuthorization(pulsar(), topicName, clientAppId(), clientAuthData());
     }
 
-    protected static void checkAuthorization(PulsarService pulsarService, DestinationName destination, String role,
+    protected static void checkAuthorization(PulsarService pulsarService, TopicName topicName, String role,
             AuthenticationDataSource authenticationData) throws RestException, Exception {
         if (!pulsarService.getConfiguration().isAuthorizationEnabled()) {
             // No enforcing of authorization policies
             return;
         }
         // get zk policy manager
-        if (!pulsarService.getBrokerService().getAuthorizationService().canLookup(destination, role, authenticationData)) {
-            log.warn("[{}] Role {} is not allowed to lookup topic", destination, role);
+        if (!pulsarService.getBrokerService().getAuthorizationService().canLookup(topicName, role, authenticationData)) {
+            log.warn("[{}] Role {} is not allowed to lookup topic", topicName, role);
             throw new RestException(Status.UNAUTHORIZED, "Don't have permission to connect to this namespace");
         }
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
index d62ae92..aac9aa0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
@@ -22,7 +22,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.naming.ServiceUnitId;
@@ -108,11 +108,11 @@ public class NamespaceBundle implements ServiceUnitId, Comparable<NamespaceBundl
     }
 
     @Override
-    public boolean includes(DestinationName dn) {
-        if (!this.nsname.equals(dn.getNamespaceObject())) {
+    public boolean includes(TopicName topicName) {
+        if (!this.nsname.equals(topicName.getNamespaceObject())) {
             return false;
         }
-        return this.keyRange.contains(factory.getLongHashCode(dn.toString()));
+        return this.keyRange.contains(factory.getLongHashCode(topicName.toString()));
     }
 
     public String getBundleRange() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
index 29d2b26..43fb219 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
@@ -26,7 +26,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.SortedSet;
 
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 
 import com.google.common.base.Objects;
@@ -86,11 +86,11 @@ public class NamespaceBundles {
         }
     }
 
-    public NamespaceBundle findBundle(DestinationName dn) {
-        checkArgument(this.nsname.equals(dn.getNamespaceObject()));
-        long hashCode = factory.getLongHashCode(dn.toString());
+    public NamespaceBundle findBundle(TopicName topicName) {
+        checkArgument(this.nsname.equals(topicName.getNamespaceObject()));
+        long hashCode = factory.getLongHashCode(topicName.toString());
         NamespaceBundle bundle = getBundle(hashCode);
-        if (dn.getDomain().equals(DestinationDomain.non_persistent)) {
+        if (topicName.getDomain().equals(TopicDomain.non_persistent)) {
             bundle.setHasNonPersistentTopic(true);
         }
         return bundle;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
index 1deb88b..e28b226 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
@@ -107,9 +107,9 @@ public class SLAMonitoringTest {
 
         createProperty(pulsarAdmins[BROKER_COUNT - 1]);
         for (int i = 0; i < BROKER_COUNT; i++) {
-            String destination = String.format("%s/%s/%s:%s", NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster",
+            String topic = String.format("%s/%s/%s:%s", NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster",
                     pulsarServices[i].getAdvertisedAddress(), brokerWebServicePorts[i]);
-            pulsarAdmins[0].namespaces().createNamespace(destination);
+            pulsarAdmins[0].namespaces().createNamespace(topic);
         }
     }
 
@@ -177,10 +177,10 @@ public class SLAMonitoringTest {
     public void testOwnershipViaAdminAfterSetup() {
         for (int i = 0; i < BROKER_COUNT; i++) {
             try {
-                String destination = String.format("persistent://%s/%s/%s:%s/%s",
+                String topic = String.format("persistent://%s/%s/%s:%s/%s",
                         NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster", pulsarServices[i].getAdvertisedAddress(),
                         brokerWebServicePorts[i], "my-topic");
-                assertEquals(pulsarAdmins[0].lookups().lookupDestination(destination),
+                assertEquals(pulsarAdmins[0].lookups().lookupTopic(topic),
                         "pulsar://" + pulsarServices[i].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[i]);
             } catch (Exception e) {
                 e.printStackTrace();
@@ -202,15 +202,15 @@ public class SLAMonitoringTest {
             fail("Should be a able to close the broker index " + crashIndex + " Exception: " + e);
         }
 
-        String destination = String.format("persistent://%s/%s/%s:%s/%s", NamespaceService.SLA_NAMESPACE_PROPERTY,
+        String topic = String.format("persistent://%s/%s/%s:%s/%s", NamespaceService.SLA_NAMESPACE_PROPERTY,
                 "my-cluster", pulsarServices[crashIndex].getAdvertisedAddress(), brokerWebServicePorts[crashIndex], "my-topic");
 
-        log.info("Lookup for namespace {}", destination);
+        log.info("Lookup for namespace {}", topic);
 
         String broker = null;
         try {
-            broker = pulsarAdmins[BROKER_COUNT - 1].lookups().lookupDestination(destination);
-            log.info("{} Namespace is owned by {}", destination, broker);
+            broker = pulsarAdmins[BROKER_COUNT - 1].lookups().lookupTopic(topic);
+            log.info("{} Namespace is owned by {}", topic, broker);
             assertNotEquals(broker,
                     "pulsar://" + pulsarServices[crashIndex].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[crashIndex]);
         } catch (PulsarAdminException e) {
@@ -230,8 +230,8 @@ public class SLAMonitoringTest {
         }
 
         try {
-            broker = pulsarAdmins[0].lookups().lookupDestination(destination);
-            log.info("{} Namespace is re-owned by {}", destination, broker);
+            broker = pulsarAdmins[0].lookups().lookupTopic(topic);
+            log.info("{} Namespace is re-owned by {}", topic, broker);
             assertEquals(broker,
                     "pulsar://" + pulsarServices[crashIndex].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[crashIndex]);
         } catch (PulsarAdminException e) {
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 5038116..6108ade 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
@@ -65,8 +65,8 @@ import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -164,8 +164,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
 
     @DataProvider(name = "topicType")
     public Object[][] topicTypeProvider() {
-        return new Object[][] { { DestinationDomain.persistent.value() },
-                { DestinationDomain.non_persistent.value() } };
+        return new Object[][] { { TopicDomain.persistent.value() },
+                { TopicDomain.non_persistent.value() } };
     }
 
     @Test
@@ -599,7 +599,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns1"), policies);
         assertEquals(admin.namespaces().getPermissions("prop-xyz/use/ns1"), policies.auth_policies.namespace_auth);
 
-        assertEquals(admin.namespaces().getDestinations("prop-xyz/use/ns1"), Lists.newArrayList());
+        assertEquals(admin.namespaces().getTopics("prop-xyz/use/ns1"), Lists.newArrayList());
 
         admin.namespaces().revokePermissionsOnNamespace("prop-xyz/use/ns1", "my-role");
         policies.auth_policies.namespace_auth.remove("my-role");
@@ -656,7 +656,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
 
         final String persistentTopicName = "persistent://prop-xyz/use/ns1/" + topicName;
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/" + topicName, 0);
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
                 Lists.newArrayList("persistent://prop-xyz/use/ns1/" + topicName));
@@ -742,8 +742,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         assertEquals(admin.persistentTopics().getPartitionedTopicMetadata(partitionedTopicName).partitions, 4);
 
         // check if the virtual topic doesn't get created
-        List<String> destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
-        assertEquals(destinations.size(), 0);
+        List<String> topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+        assertEquals(topics.size(), 0);
 
         assertEquals(
                 admin.persistentTopics().getPartitionedTopicMetadata("persistent://prop-xyz/use/ns1/ds2").partitions,
@@ -836,8 +836,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
 
         producer = client.createProducer(partitionedTopicName);
 
-        destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
-        assertEquals(destinations.size(), 4);
+        topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+        assertEquals(topics.size(), 4);
 
         try {
             admin.persistentTopics().deletePartitionedTopic(partitionedTopicName);
@@ -876,12 +876,12 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         admin.namespaces().createNamespace("prop-xyz/use/ns1-bundles", numBundles);
 
         // since we have 2 brokers running, we try to let both of them acquire bundle ownership
-        admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds1");
-        admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds2");
-        admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds3");
-        admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds4");
+        admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds1");
+        admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds2");
+        admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds3");
+        admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds4");
 
-        assertEquals(admin.namespaces().getDestinations("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
+        assertEquals(admin.namespaces().getTopics("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
 
         admin.namespaces().deleteNamespace("prop-xyz/use/ns1-bundles");
         assertEquals(admin.namespaces().getNamespaces("prop-xyz", "use"), Lists.newArrayList());
@@ -889,7 +889,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
 
     @Test
     public void testNamespaceSplitBundle() throws Exception {
-        // Force to create a destination
+        // Force to create a topic
         final String namespace = "prop-xyz/use/ns1";
         final String topicName = (new StringBuilder("persistent://")).append(namespace).append("/ds2").toString();
         Producer producer = pulsarClient.createProducer(topicName);
@@ -917,7 +917,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     public void testNamespaceUnloadBundle() throws Exception {
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
 
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/ds2", 0);
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
                 Lists.newArrayList("persistent://prop-xyz/use/ns1/ds2"));
@@ -975,7 +975,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
 
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
 
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1-bundles/ds2", 0);
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1-bundles"),
                 Lists.newArrayList("persistent://prop-xyz/use/ns1-bundles/ds2"));
@@ -994,7 +994,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         }
 
         NamespaceBundle bundle = (NamespaceBundle) pulsar.getNamespaceService()
-                .getBundle(DestinationName.get("persistent://prop-xyz/use/ns1-bundles/ds2"));
+                .getBundle(TopicName.get("persistent://prop-xyz/use/ns1-bundles/ds2"));
 
         consumer.close();
         producer.close();
@@ -1164,7 +1164,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     }
 
     @Test
-    public void statsOnNonExistingDestinations() throws Exception {
+    public void statsOnNonExistingTopics() throws Exception {
         try {
             admin.persistentTopics().getStats("persistent://prop-xyz/use/ns1/ghostTopic");
             fail("The topic doesn't exist");
@@ -1384,8 +1384,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         conf.setSubscriptionType(SubscriptionType.Exclusive);
         Consumer consumer = pulsarClient.subscribe(topicName, "my-sub", conf);
 
-        List<String> destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
-        assertEquals(destinations.size(), 4);
+        List<String> topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+        assertEquals(topics.size(), 4);
 
         assertEquals(admin.persistentTopics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));
 
@@ -1427,7 +1427,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
 
         String topicName = "persistent://prop-xyz/use/ns1/invalidcursorreset";
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnPersistentTopic(topicName, 0);
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList(topicName));
 
@@ -1504,7 +1504,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     @Test
     public void testPersistentTopicsExpireMessages() throws Exception {
 
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/ds2", 0);
         assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
                 Lists.newArrayList("persistent://prop-xyz/use/ns1/ds2"));
@@ -1617,13 +1617,13 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     @Test(dataProvider = "topicName")
     public void testPulsarAdminForUriAndUrlEncoding(String topicName) throws Exception {
         final String ns1 = "prop-xyz/use/ns1";
-        final String dn1 = "persistent://" + ns1 + "/" + topicName;
+        final String topic1 = "persistent://" + ns1 + "/" + topicName;
         final String urlEncodedTopic = Codec.encode(topicName);
         final String uriEncodedTopic = urlEncodedTopic.replaceAll("\\+", "%20");
         final int numOfPartitions = 4;
-        admin.persistentTopics().createPartitionedTopic(dn1, numOfPartitions);
+        admin.persistentTopics().createPartitionedTopic(topic1, numOfPartitions);
         // Create a consumer to get stats on this topic
-        pulsarClient.subscribe(dn1, "my-subscriber-name", new ConsumerConfiguration());
+        pulsarClient.subscribe(topic1, "my-subscriber-name", new ConsumerConfiguration());
 
         PersistentTopicsImpl persistent = (PersistentTopicsImpl) admin.persistentTopics();
         Field field = PersistentTopicsImpl.class.getDeclaredField("persistentTopics");
@@ -1741,7 +1741,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     }
 
     @Test
-    public void testDestinationBundleRangeLookup() throws PulsarAdminException, PulsarServerException, Exception {
+    public void testTopicBundleRangeLookup() throws PulsarAdminException, PulsarServerException, Exception {
         admin.clusters().createCluster("usw", new ClusterData());
         PropertyAdmin propertyAdmin = new PropertyAdmin(Lists.newArrayList("role1", "role2"),
                 Sets.newHashSet("use", "usw"));
@@ -1753,7 +1753,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
         final String topicName = "persistent://prop-xyz/use/getBundleNs/topic1";
         String bundleRange = admin.lookups().getBundleRange(topicName);
         assertEquals(bundleRange,
-                pulsar.getNamespaceService().getBundle(DestinationName.get(topicName)).getBundleRange());
+                pulsar.getNamespaceService().getBundle(TopicName.get(topicName)).getBundleRange());
     }
 
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
index bbbb42e..711d125 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
@@ -54,8 +54,8 @@ import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.impl.MessageIdImpl;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.FailureDomain;
 import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
@@ -79,8 +79,6 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStr
 
 public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
 
-    private static final Logger LOG = LoggerFactory.getLogger(AdminApiTest2.class);
-
     private MockedPulsarService mockPulsarSetup;
 
 
@@ -111,8 +109,8 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
 
     @DataProvider(name = "topicType")
     public Object[][] topicTypeProvider() {
-        return new Object[][] { { DestinationDomain.persistent.value() },
-                { DestinationDomain.non_persistent.value() } };
+        return new Object[][] { { TopicDomain.persistent.value() },
+                { TopicDomain.non_persistent.value() } };
     }
 
     @DataProvider(name = "namespaceNames")
@@ -170,7 +168,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
         assertEquals(admin.persistentTopics().getPartitionedTopicMetadata(partitionedTopicName).partitions,
                 newPartitions);
         // (2) No Msg loss: verify new partitions have the same existing subscription names
-        final String newPartitionTopicName = DestinationName.get(partitionedTopicName).getPartition(startPartitions + 1)
+        final String newPartitionTopicName = TopicName.get(partitionedTopicName).getPartition(startPartitions + 1)
                 .toString();
 
         // (3) produce messages to all partitions including newly created partitions (RoundRobin)
@@ -212,7 +210,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
         assertEquals(topicStats.partitions.keySet(), partitionSet);
         for (int i = 0; i < newPartitions; i++) {
             PersistentTopicStats partitionStats = topicStats.partitions
-                    .get(DestinationName.get(partitionedTopicName).getPartition(i).toString());
+                    .get(TopicName.get(partitionedTopicName).getPartition(i).toString());
             assertEquals(partitionStats.publishers.size(), 1);
             assertEquals(partitionStats.subscriptions.get(subName2).consumers.size(), 1);
             assertEquals(partitionStats.subscriptions.get(subName2).msgBacklog, 2, 1);
@@ -235,7 +233,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
         final String topicName = "nonPersistentTopic";
 
         final String persistentTopicName = "non-persistent://prop-xyz/use/ns1/" + topicName;
-        // Force to create a destination
+        // Force to create a topic
         publishMessagesOnTopic("non-persistent://prop-xyz/use/ns1/" + topicName, 0, 0);
 
         // create consumer and subscription
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
index dec7768..8e66aca 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
@@ -584,8 +584,8 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
         assertNotNull(allocatorStats);
         Map<String, Map<String, PendingBookieOpsStats>> bookieOpsStats = brokerStats.getPendingBookieOpsStats();
         assertTrue(bookieOpsStats.isEmpty());
-        StreamingOutput destination = brokerStats.getDestinations2();
-        assertNotNull(destination);
+        StreamingOutput topic = brokerStats.getTopics2();
+        assertNotNull(topic);
         try {
             brokerStats.getBrokerResourceAvailability("prop", "use", "ns2");
             fail("should have failed as ModularLoadManager doesn't support it");
@@ -600,7 +600,7 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
         final String property = "prop-xyz";
         final String cluster = "use";
         final String namespace = "ns";
-        final String destination = "ds1";
+        final String topic = "ds1";
         Policies policies = new Policies();
         doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
         doReturn("client-id").when(resourceQuotas).clientAppId();
@@ -613,11 +613,11 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
 
         List<String> list = persistentTopics.getList(property, cluster, namespace);
         assertTrue(list.isEmpty());
-        // create destination
+        // create topic
         assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList());
-        persistentTopics.createPartitionedTopic(property, cluster, namespace, destination, 5, false);
+        persistentTopics.createPartitionedTopic(property, cluster, namespace, topic, 5, false);
         assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists
-                .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
+                .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, topic)));
 
         CountDownLatch notificationLatch = new CountDownLatch(2);
         configurationCache.policiesCache().registerListener((path, data, stat) -> {
@@ -627,19 +627,19 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
         // grant permission
         final Set<AuthAction> actions = Sets.newHashSet(AuthAction.produce);
         final String role = "test-role";
-        persistentTopics.grantPermissionsOnDestination(property, cluster, namespace, destination, role, actions);
+        persistentTopics.grantPermissionsOnTopic(property, cluster, namespace, topic, role, actions);
         // verify permission
-        Map<String, Set<AuthAction>> permission = persistentTopics.getPermissionsOnDestination(property, cluster,
-                namespace, destination);
+        Map<String, Set<AuthAction>> permission = persistentTopics.getPermissionsOnTopic(property, cluster,
+                namespace, topic);
         assertEquals(permission.get(role), actions);
         // remove permission
-        persistentTopics.revokePermissionsOnDestination(property, cluster, namespace, destination, role);
+        persistentTopics.revokePermissionsOnTopic(property, cluster, namespace, topic, role);
 
         // Wait for cache to be updated
         notificationLatch.await();
 
         // verify removed permission
-        permission = persistentTopics.getPermissionsOnDestination(property, cluster, namespace, destination);
+        permission = persistentTopics.getPermissionsOnTopic(property, cluster, namespace, topic);
         assertTrue(permission.isEmpty());
     }
 
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
index 6f759d8..c6f277d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
@@ -28,7 +28,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException;
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -99,7 +99,7 @@ public class CreateSubscriptionTest extends MockedPulsarServiceBaseTest {
 
         for (int i = 0; i < 10; i++) {
             assertEquals(
-                    admin.persistentTopics().getSubscriptions(DestinationName.get(topic).getPartition(i).toString()),
+                    admin.persistentTopics().getSubscriptions(TopicName.get(topic).getPartition(i).toString()),
                     Lists.newArrayList("sub-1"));
         }
     }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
index 779466a..e001ba0 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
@@ -23,7 +23,7 @@ import static org.testng.Assert.assertEquals;
 import org.apache.pulsar.broker.admin.AdminApiTest.MockedPulsarService;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.client.api.Consumer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
 import org.testng.annotations.AfterMethod;
@@ -87,7 +87,7 @@ public class IncrementPartitionsTest extends MockedPulsarServiceBaseTest {
 
         assertEquals(
                 admin.persistentTopics()
-                        .getSubscriptions(DestinationName.get(partitionedTopicName).getPartition(15).toString()),
+                        .getSubscriptions(TopicName.get(partitionedTopicName).getPartition(15).toString()),
                 Lists.newArrayList("sub-1"));
 
         consumer.close();
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 f9ff652..16ca86c 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
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.web.PulsarWebResource;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundles;
 import org.apache.pulsar.common.naming.NamespaceName;
@@ -598,7 +598,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
         }
 
         NamespaceName testNs = this.testLocalNamespaces.get(1);
-        DestinationName topicName = DestinationName.get(testNs.getPersistentTopicName("my-topic"));
+        TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic"));
         ZkUtils.createFullPathOptimistic(mockZookKeeper, "/managed-ledgers/" + topicName.getPersistenceNamingEncoding(),
                 new byte[0], null, null);
 
@@ -640,7 +640,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
 
         // delete the topic from ZK
         mockZookKeeper.delete("/managed-ledgers/" + topicName.getPersistenceNamingEncoding(), -1);
-        // ensure refreshed destination list in the cache
+        // ensure refreshed topics list in the cache
         pulsar.getLocalZkCacheService().managedLedgerListCache().clearTree();
         // setup ownership to localhost
         doReturn(Optional.of(localWebServiceUrl)).when(nsSvc).getWebServiceUrl(testNs, false, false, false);
@@ -1047,7 +1047,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
     }
 
     @Test
-    public void testValidateDestinationOwnership() throws Exception {
+    public void testValidateTopicOwnership() throws Exception {
         try {
             URL localWebServiceUrl = new URL(pulsar.getWebServiceAddress());
             String bundledNsLocal = "test-bundled-namespace-1";
@@ -1059,7 +1059,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
             Field ownership = NamespaceService.class.getDeclaredField("ownershipCache");
             ownership.setAccessible(true);
             ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
-            DestinationName topicName = DestinationName.get(testNs.getPersistentTopicName("my-topic"));
+            TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic"));
             PersistentTopics topics = spy(new PersistentTopics());
             topics.setServletContext(new MockServletContext());
             topics.setPulsar(pulsar);
@@ -1069,9 +1069,9 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
             doReturn("persistent").when(topics).domain();
 
             try {
-                topics.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
+                topics.validateTopicName(topicName.getProperty(), topicName.getCluster(),
                         topicName.getNamespacePortion(), topicName.getEncodedLocalName());
-                topics.validateAdminOperationOnDestination(false);
+                topics.validateAdminOperationOnTopic(false);
             } catch (RestException e) {
                 fail("validateAdminAccessOnProperty failed");
             }
@@ -1100,17 +1100,17 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
 
         // (1) Force topic creation and namespace being loaded
         final String topicName = "persistent://" + namespace + "/my-topic";
-        DestinationName destination = DestinationName.get(topicName);
+        TopicName topic = TopicName.get(topicName);
 
         Producer producer = pulsarClient.createProducer(topicName);
         producer.close();
-        NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(destination);
+        NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(topic);
         // (2) Delete topic
         admin.persistentTopics().delete(topicName);
         // (3) Delete ns
         admin.namespaces().deleteNamespace(namespace);
         // (4) check bundle
-        NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(destination);
+        NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(topic);
         assertNotEquals(bundle1.getBundleRange(), bundle2.getBundleRange());
         // returns full bundle if policies not present
         assertEquals("0x00000000_0xffffffff", bundle2.getBundleRange());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
index c95fbd7..4f0baf6 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
@@ -24,7 +24,7 @@ import static org.testng.Assert.fail;
 import java.util.EnumSet;
 
 import org.apache.pulsar.broker.authorization.AuthorizationService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -63,7 +63,7 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
     void simple() throws Exception {
         AuthorizationService auth = pulsar.getBrokerService().getAuthorizationService();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
 
         admin.clusters().updateCluster("c1", new ClusterData());
         admin.properties().createProperty("p1", new PropertyAdmin(Lists.newArrayList("role1"), Sets.newHashSet("c1")));
@@ -71,120 +71,120 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
         admin.namespaces().createNamespace("p1/c1/ns1");
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.of(AuthAction.produce));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
 
         admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role",
                 EnumSet.of(AuthAction.consume));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.allOf(AuthAction.class));
         waitForChange();
 
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
 
         // test for wildcard
 
         // namespace prefix match
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my.role.*", EnumSet.of(AuthAction.produce));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
 
         // namespace suffix match
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "*.role.my", EnumSet.of(AuthAction.consume));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
 
         // revoke for next test
         admin.namespaces().revokePermissionsOnNamespace("p1/c1/ns1", "my.role.*");
         admin.namespaces().revokePermissionsOnNamespace("p1/c1/ns1", "*.role.my");
         waitForChange();
 
-        // destination prefix match
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
+        // topic prefix match
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
 
         admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds1", "my.*",
                 EnumSet.of(AuthAction.produce));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
-
-        // destination suffix match
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
+
+        // topic suffix match
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
 
         admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds1", "*.my",
                 EnumSet.of(AuthAction.consume));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
 
         admin.persistentTopics().revokePermissions("persistent://p1/c1/ns1/ds1", "my.*");
         admin.persistentTopics().revokePermissions("persistent://p1/c1/ns1/ds1", "*.my");
@@ -194,20 +194,20 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
         admin.namespaces().setSubscriptionAuthMode("p1/c1/ns1", SubscriptionAuthMode.Prefix);
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null), true);
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null), true);
         try {
-            assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "sub1"), false);
+            assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "sub1"), false);
             fail();
         } catch (Exception e) {}
         try {
-            assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "sub2"), false);
+            assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "sub2"), false);
             fail();
         } catch (Exception e) {}
 
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "role1-sub1"), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2"), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1"), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "role1-sub1"), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2"), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1"), true);
 
         admin.namespaces().deleteNamespace("p1/c1/ns1");
         admin.properties().deleteProperty("p1");
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 1cf6822..9ebba22 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
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.admin.internal.NamespacesImpl;
 import org.apache.pulsar.client.api.Authentication;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData;
 import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType;
@@ -223,9 +223,9 @@ public class LoadBalancerTest {
                     brokerCount += entry.getValue().size();
                 }
                 assertEquals(brokerCount, BROKER_COUNT);
-                DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns/test-topic");
+                TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns/test-topic");
                 ResourceUnit found = pulsarServices[i].getLoadManager().get()
-                        .getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(fqdn)).get();
+                        .getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(topicName)).get();
                 assertTrue(found != null);
             }
         } catch (InterruptedException | KeeperException e) {
@@ -262,9 +262,9 @@ public class LoadBalancerTest {
         int totalNamespaces = 200;
         Map<String, Integer> namespaceOwner = new HashMap<>();
         for (int i = 0; i < totalNamespaces; i++) {
-            DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
+            TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
             ResourceUnit found = pulsarServices[0].getLoadManager().get()
-                    .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)).get();
+                    .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(topicName)).get();
             if (namespaceOwner.containsKey(found.getResourceId())) {
                 namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
             } else {
@@ -351,10 +351,10 @@ public class LoadBalancerTest {
     /*
      * Pre-publish load report to ZK, each broker has: - Difference memory capacity, for the first 3 brokers memory is
      * bottleneck, for the 4/5th brokers CPU become bottleneck since memory is big enough - already has some bundles
-     * assigned Check the distribution of new destinations is roughly consistent (with <10% variation) with the ranking
+     * assigned Check the distribution of new topics is roughly consistent (with <10% variation) with the ranking
      */
     @Test
-    public void testDestinationAssignmentWithExistingBundles() throws Exception {
+    public void testTopicAssignmentWithExistingBundles() throws Exception {
         for (int i = 0; i < BROKER_COUNT; i++) {
             ResourceQuota defaultQuota = new ResourceQuota();
             defaultQuota.setMsgRateIn(20);
@@ -402,9 +402,9 @@ public class LoadBalancerTest {
         int[] expectedAssignments = new int[] { 17, 34, 51, 68, 85 };
         Map<String, Integer> namespaceOwner = new HashMap<>();
         for (int i = 0; i < totalNamespaces; i++) {
-            DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
+            TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
             ResourceUnit found = pulsarServices[0].getLoadManager().get()
-                    .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)).get();
+                    .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(topicName)).get();
             if (namespaceOwner.containsKey(found.getResourceId())) {
                 namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
             } else {
@@ -422,7 +422,7 @@ public class LoadBalancerTest {
 
             long expectedValue = expectedAssignments[i];
             double variation = Math.abs(actualValue - expectedValue) * 100.0 / expectedValue;
-            log.info("Destination assignment - {}, actual: {}, expected baseline: {}, variation: {}/%",
+            log.info("Topic assignment - {}, actual: {}, expected baseline: {}, variation: {}/%",
                     lookupAddresses[i], actualValue, expectedValue, String.format("%.2f", variation));
             assertTrue(variation < expectedMaxVariation);
         }
@@ -951,7 +951,7 @@ public class LoadBalancerTest {
         Map<String, Integer> namespaceOwner = new HashMap<String, Integer>();
         for (int i = 0; i < totalNamespaces; i++) {
             ResourceUnit found = loadManager
-                    .getLeastLoaded(DestinationName.get("persistent://pulsar/use/primary-ns/topic" + i)).get();
+                    .getLeastLoaded(TopicName.get("persistent://pulsar/use/primary-ns/topic" + i)).get();
             if (namespaceOwner.containsKey(found.getResourceId())) {
                 namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
             } else {
@@ -1012,7 +1012,7 @@ public class LoadBalancerTest {
         Map<String, Integer> namespaceOwner = new HashMap<String, Integer>();
         for (int i = 0; i < totalNamespaces; i++) {
             ResourceUnit found = loadManager
-                    .getLeastLoaded(DestinationName.get("persistent://pulsar/use/primary-ns/topic-" + i)).get();
+                    .getLeastLoaded(TopicName.get("persistent://pulsar/use/primary-ns/topic-" + i)).get();
             if (namespaceOwner.containsKey(found.getResourceId())) {
                 namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
             } else {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
similarity index 92%
rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java
rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
index 645fff9..6eefc8f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
@@ -43,14 +43,14 @@ import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.authorization.AuthorizationService;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.broker.lookup.DestinationLookup;
+import org.apache.pulsar.broker.lookup.TopicLookup;
 import org.apache.pulsar.broker.lookup.NamespaceData;
 import org.apache.pulsar.broker.lookup.RedirectData;
 import org.apache.pulsar.broker.namespace.NamespaceService;
 import org.apache.pulsar.broker.service.BrokerService;
 import org.apache.pulsar.broker.web.PulsarWebResource;
 import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationDomain;
+import org.apache.pulsar.common.naming.TopicDomain;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.zookeeper.ZooKeeperChildrenCache;
@@ -66,7 +66,7 @@ import com.google.common.collect.Lists;
  *
  *
  */
-public class HttpDestinationLookupv2Test {
+public class HttpTopicLookupv2Test {
 
     private PulsarService pulsar;
     private NamespaceService ns;
@@ -119,7 +119,7 @@ public class HttpDestinationLookupv2Test {
     @Test
     public void crossColoLookup() throws Exception {
 
-        DestinationLookup destLookup = spy(new DestinationLookup());
+        TopicLookup destLookup = spy(new TopicLookup());
         doReturn(false).when(destLookup).isRequestHttps();
         destLookup.setPulsar(pulsar);
         doReturn("null").when(destLookup).clientAppId();
@@ -132,7 +132,7 @@ public class HttpDestinationLookupv2Test {
         doReturn(true).when(config).isAuthorizationEnabled();
 
         AsyncResponse asyncResponse = mock(AsyncResponse.class);
-        destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
+        destLookup.lookupTopicAsync(TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
                 asyncResponse);
 
         ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -149,7 +149,7 @@ public class HttpDestinationLookupv2Test {
         BrokerService brokerService = pulsar.getBrokerService();
         doReturn(new Semaphore(0)).when(brokerService).getLookupRequestSemaphore();
 
-        DestinationLookup destLookup = spy(new DestinationLookup());
+        TopicLookup destLookup = spy(new TopicLookup());
         doReturn(false).when(destLookup).isRequestHttps();
         destLookup.setPulsar(pulsar);
         doReturn("null").when(destLookup).clientAppId();
@@ -162,7 +162,7 @@ public class HttpDestinationLookupv2Test {
         doReturn(true).when(config).isAuthorizationEnabled();
 
         AsyncResponse asyncResponse1 = mock(AsyncResponse.class);
-        destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
+        destLookup.lookupTopicAsync(TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
                 asyncResponse1);
 
         ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -187,7 +187,7 @@ public class HttpDestinationLookupv2Test {
         doReturn(Optional.of(policies2)).when(policiesCache)
                 .get(AdminResource.path(POLICIES, property, cluster, ns2));
 
-        DestinationLookup destLookup = spy(new DestinationLookup());
+        TopicLookup destLookup = spy(new TopicLookup());
         doReturn(false).when(destLookup).isRequestHttps();
         destLookup.setPulsar(pulsar);
         doReturn("null").when(destLookup).clientAppId();
@@ -198,7 +198,7 @@ public class HttpDestinationLookupv2Test {
         doReturn(false).when(config).isAuthorizationEnabled();
 
         AsyncResponse asyncResponse = mock(AsyncResponse.class);
-        destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), property, cluster, ns1, "empty-cluster",
+        destLookup.lookupTopicAsync(TopicDomain.persistent.value(), property, cluster, ns1, "empty-cluster",
                 false, asyncResponse);
 
         ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -206,7 +206,7 @@ public class HttpDestinationLookupv2Test {
         assertEquals(arg.getValue().getClass(), RestException.class);
 
         AsyncResponse asyncResponse2 = mock(AsyncResponse.class);
-        destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), property, cluster, ns2,
+        destLookup.lookupTopicAsync(TopicDomain.persistent.value(), property, cluster, ns2,
                 "invalid-localCluster", false, asyncResponse2);
         ArgumentCaptor<Throwable> arg2 = ArgumentCaptor.forClass(Throwable.class);
         verify(asyncResponse2).resume(arg2.capture());
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 f20a877..659361c 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
@@ -56,7 +56,7 @@ import org.apache.pulsar.broker.service.Topic;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.client.api.Consumer;
 import org.apache.pulsar.client.api.ConsumerConfiguration;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -108,9 +108,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
         NamespaceService namespaceService = pulsar.getNamespaceService();
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
-        NamespaceBundle originalBundle = bundles.findBundle(dn);
+        NamespaceBundle originalBundle = bundles.findBundle(topicName);
 
         // Split bundle and take ownership of split bundles
         CompletableFuture<Void> result = namespaceService.splitAndOwnBundle(originalBundle, false);
@@ -178,15 +178,15 @@ public class NamespaceServiceTest extends BrokerTestBase {
 
         NamespaceService namespaceService = pulsar.getNamespaceService();
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
-        NamespaceBundle originalBundle = bundles.findBundle(dn);
+        NamespaceBundle originalBundle = bundles.findBundle(topicName);
 
-        PersistentTopic topic = new PersistentTopic(dn.toString(), ledger, pulsar.getBrokerService());
+        PersistentTopic topic = new PersistentTopic(topicName.toString(), ledger, pulsar.getBrokerService());
         Method method = pulsar.getBrokerService().getClass().getDeclaredMethod("addTopicToStatsMaps",
-                DestinationName.class, Topic.class);
+                TopicName.class, Topic.class);
         method.setAccessible(true);
-        method.invoke(pulsar.getBrokerService(), dn, topic);
+        method.invoke(pulsar.getBrokerService(), topicName, topic);
         String nspace = originalBundle.getNamespaceObject().toString();
         List<Topic> list = this.pulsar.getBrokerService().getAllTopicsFromNamespaceBundle(nspace,
                 originalBundle.toString());
@@ -210,7 +210,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
         }
 
         // status-map should be updated with new split bundles
-        NamespaceBundle splitBundle = pulsar.getNamespaceService().getBundle(dn);
+        NamespaceBundle splitBundle = pulsar.getNamespaceService().getBundle(topicName);
         assertTrue(!CollectionUtils.isEmpty(
                 this.pulsar.getBrokerService().getAllTopicsFromNamespaceBundle(nspace, splitBundle.toString())));
 
@@ -231,9 +231,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
 
         NamespaceService namespaceService = pulsar.getNamespaceService();
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
-        NamespaceBundle originalBundle = bundles.findBundle(dn);
+        NamespaceBundle originalBundle = bundles.findBundle(topicName);
 
         assertFalse(namespaceService.isNamespaceBundleDisabled(originalBundle));
 
@@ -283,7 +283,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
                 return result;
             }
         }).when(spyTopic).close();
-        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
         try {
             pulsar.getNamespaceService().unloadNamespaceBundle(bundle);
         } catch (Exception e) {
@@ -322,7 +322,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
                 return new CompletableFuture<Void>();
             }
         }).when(spyTopic).close();
-        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
 
         // try to unload bundle whose topic will be stuck
         pulsar.getNamespaceService().unloadNamespaceBundle(bundle, 1, TimeUnit.SECONDS);
@@ -381,9 +381,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
         ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
         NamespaceService namespaceService = pulsar.getNamespaceService();
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
-        NamespaceBundle originalBundle = bundles.findBundle(dn);
+        NamespaceBundle originalBundle = bundles.findBundle(topicName);
 
         // Split bundle and take ownership of split bundles
         CompletableFuture<Void> result = namespaceService.splitAndOwnBundle(originalBundle, false);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
index f5711eb..d688b7f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
@@ -150,12 +150,12 @@ public class BrokerBkEnsemblesTests {
 
         admin.namespaces().createNamespace(ns1);
 
-        final String dn1 = "persistent://" + ns1 + "/my-topic";
+        final String topic1 = "persistent://" + ns1 + "/my-topic";
 
         // (1) create topic
         // publish and ack messages so, cursor can create cursor-ledger and update metadata
-        Consumer consumer = client.subscribe(dn1, "my-subscriber-name");
-        Producer producer = client.createProducer(dn1);
+        Consumer consumer = client.subscribe(topic1, "my-subscriber-name");
+        Producer producer = client.createProducer(topic1);
         for (int i = 0; i < 10; i++) {
             String message = "my-message-" + i;
             producer.send(message.getBytes());
@@ -166,7 +166,7 @@ public class BrokerBkEnsemblesTests {
             consumer.acknowledge(msg);
         }
 
-        PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+        PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
         ManagedCursorImpl cursor = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next();
         retryStrategically((test) -> cursor.getState().equals("Open"), 5, 100);
 
@@ -178,7 +178,7 @@ public class BrokerBkEnsemblesTests {
         // (3) remove topic and managed-ledger from broker which means topic is not closed gracefully
         consumer.close();
         producer.close();
-        pulsar.getBrokerService().removeTopicFromCache(dn1);
+        pulsar.getBrokerService().removeTopicFromCache(topic1);
         ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory();
         Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers");
         field.setAccessible(true);
@@ -189,8 +189,8 @@ public class BrokerBkEnsemblesTests {
 
         // (4) Recreate topic
         // publish and ack messages so, cursor can create cursor-ledger and update metadata
-        consumer = client.subscribe(dn1, "my-subscriber-name");
-        producer = client.createProducer(dn1);
+        consumer = client.subscribe(topic1, "my-subscriber-name");
+        producer = client.createProducer(topic1);
         for (int i = 0; i < 10; i++) {
             String message = "my-message-" + i;
             producer.send(message.getBytes());
@@ -201,7 +201,7 @@ public class BrokerBkEnsemblesTests {
         }
 
         // (5) Broker should create new cursor-ledger and remove old cursor-ledger
-        topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+        topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
         final ManagedCursorImpl cursor1 = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next();
         retryStrategically((test) -> cursor1.getState().equals("Open"), 5, 100);
         long newCursorLedgerId = cursor1.getCursorLedger();
@@ -244,14 +244,14 @@ public class BrokerBkEnsemblesTests {
 
         admin.namespaces().createNamespace(ns1);
 
-        final String dn1 = "persistent://" + ns1 + "/my-topic";
+        final String topic1 = "persistent://" + ns1 + "/my-topic";
 
         // Create subscription
         ConsumerConfiguration consumerConfig = new ConsumerConfiguration();
         consumerConfig.setReceiverQueueSize(5);
-        Consumer consumer = client.subscribe(dn1, "my-subscriber-name", consumerConfig);
+        Consumer consumer = client.subscribe(topic1, "my-subscriber-name", consumerConfig);
 
-        PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+        PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
         ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger();
         ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next();
         Field configField = ManagedCursorImpl.class.getDeclaredField("config");
@@ -267,7 +267,7 @@ public class BrokerBkEnsemblesTests {
         BookKeeper bookKeeper = (BookKeeper) bookKeeperField.get(ml);
 
         // (1) publish messages in 5 data-ledgers each with 20 entries under managed-ledger
-        Producer producer = client.createProducer(dn1);
+        Producer producer = client.createProducer(topic1);
         for (int i = 0; i < totalMessages; i++) {
             String message = "my-message-" + i;
             producer.send(message.getBytes());
@@ -294,7 +294,7 @@ public class BrokerBkEnsemblesTests {
 
         // clean managed-ledger and recreate topic to clean any data from the cache
         producer.close();
-        pulsar.getBrokerService().removeTopicFromCache(dn1);
+        pulsar.getBrokerService().removeTopicFromCache(topic1);
         ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory();
         Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers");
         field.setAccessible(true);
@@ -306,7 +306,7 @@ public class BrokerBkEnsemblesTests {
         // (3) consumer will fail to consume any message as first data-ledger is non-recoverable
         Message msg = null;
         // start consuming message
-        consumer = client.subscribe(dn1, "my-subscriber-name");
+        consumer = client.subscribe(topic1, "my-subscriber-name");
         msg = consumer.receive(1, TimeUnit.SECONDS);
         Assert.assertNull(msg);
         consumer.close();
@@ -317,7 +317,7 @@ public class BrokerBkEnsemblesTests {
         retryStrategically((test) -> config.isAutoSkipNonRecoverableData(), 5, 100);
 
         // (5) consumer will be able to consume 20 messages from last non-deleted ledger
-        consumer = client.subscribe(dn1, "my-subscriber-name");
+        consumer = client.subscribe(topic1, "my-subscriber-name");
         for (int i = 0; i < entriesPerLedger; i++) {
             msg = consumer.receive(5, TimeUnit.SECONDS);
             System.out.println(i);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
index 1c2b9b6..f0c28ab 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
@@ -63,7 +63,7 @@ import org.apache.pulsar.client.api.Producer;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.impl.auth.AuthenticationTls;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.BundlesData;
 import org.apache.pulsar.common.policies.data.LocalPolicies;
@@ -114,7 +114,7 @@ public class BrokerServiceTest extends BrokerTestBase {
         });
         latch1.await();
 
-        admin.lookups().lookupDestination(topic);
+        admin.lookups().lookupTopic(topic);
 
         final CountDownLatch latch2 = new CountDownLatch(1);
         service.getTopic(topic).thenAccept(t -> {
@@ -371,15 +371,15 @@ public class BrokerServiceTest extends BrokerTestBase {
         }
 
         rolloverPerIntervalStats();
-        JsonObject destinationStats = brokerStatsClient.getDestinations();
-        assertEquals(destinationStats.size(), 2, destinationStats.toString());
+        JsonObject topicStats = brokerStatsClient.getTopics();
+        assertEquals(topicStats.size(), 2, topicStats.toString());
 
         for (String ns : nsList) {
-            JsonObject nsObject = destinationStats.getAsJsonObject(ns);
-            List<String> topicList = admin.namespaces().getDestinations(ns);
+            JsonObject nsObject = topicStats.getAsJsonObject(ns);
+            List<String> topicList = admin.namespaces().getTopics(ns);
             for (String topic : topicList) {
                 NamespaceBundle bundle = (NamespaceBundle) pulsar.getNamespaceService()
-                        .getBundle(DestinationName.get(topic));
+                        .getBundle(TopicName.get(topic));
                 JsonObject bundleObject = nsObject.getAsJsonObject(bundle.getBundleRange());
                 JsonObject topicObject = bundleObject.getAsJsonObject("persistent");
                 AtomicBoolean topicPresent = new AtomicBoolean();
@@ -396,8 +396,8 @@ public class BrokerServiceTest extends BrokerTestBase {
             producer.close();
         }
         for (String ns : nsList) {
-            List<String> destinations = admin.namespaces().getDestinations(ns);
-            for (String dest : destinations) {
+            List<String> topics = admin.namespaces().getTopics(ns);
+            for (String dest : topics) {
                 admin.persistentTopics().delete(dest);
             }
             admin.namespaces().deleteNamespace(ns);
@@ -761,12 +761,12 @@ public class BrokerServiceTest extends BrokerTestBase {
 
         // own namespace bundle
         final String topicName = "persistent://" + namespace + "/my-topic";
-        DestinationName destination = DestinationName.get(topicName);
+        TopicName topic = TopicName.get(topicName);
         Producer producer = pulsarClient.createProducer(topicName);
         producer.close();
 
         // disable namespace-bundle
-        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(destination);
+        NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(topic);
         pulsar.getNamespaceService().getOwnershipCache().updateBundleState(bundle, false);
 
         // try to create topic which should fail as bundle is disable
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
index 2426997..e25581f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
@@ -72,7 +72,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
 import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream;
@@ -175,7 +175,7 @@ public class PersistentDispatcherFailoverConsumerTest {
         NamespaceService nsSvc = mock(NamespaceService.class);
         doReturn(nsSvc).when(pulsar).getNamespaceService();
         doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
-        doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+        doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
 
         setupMLAsyncCallbackMocks();
 
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
index 7bc3141..fa4da0b 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
@@ -47,7 +47,7 @@ import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
 import org.apache.pulsar.client.impl.MessageIdImpl;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.FutureUtil;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -326,7 +326,7 @@ public class PersistentFailoverE2ETest extends BrokerTestBase {
         int numPartitions = 4;
 
         final String topicName = "persistent://prop/use/ns-abc/failover-topic2";
-        final DestinationName destName = DestinationName.get(topicName);
+        final TopicName destName = TopicName.get(topicName);
         final String subName = "sub1";
         final int numMsgs = 100;
         Set<String> uniqueMessages = new HashSet<>();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
index d694072..eda685a 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.ServerCnx;
 import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.common.api.proto.PulsarApi;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
 import org.apache.zookeeper.ZooKeeper;
@@ -101,7 +101,7 @@ public class PersistentTopicConcurrentTest extends MockedBookKeeperTestCase {
         NamespaceService nsSvc = mock(NamespaceService.class);
         doReturn(nsSvc).when(pulsar).getNamespaceService();
         doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
-        doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+        doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
 
         final List<Position> addedEntries = Lists.newArrayList();
 
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
index 3ce76d1..ad46179 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
@@ -74,7 +74,7 @@ import org.apache.pulsar.client.impl.ConsumerImpl;
 import org.apache.pulsar.client.impl.MessageIdImpl;
 import org.apache.pulsar.client.impl.ProducerImpl;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
 import org.apache.pulsar.common.stats.Metrics;
 import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet;
@@ -548,20 +548,20 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
 
     @Test(enabled = false)
     public void testUnloadNamespace() throws Exception {
-        String topicName = "persistent://prop/use/ns-abc/topic-9";
-        DestinationName destinationName = DestinationName.get(topicName);
-        pulsarClient.createProducer(topicName);
+        String topic = "persistent://prop/use/ns-abc/topic-9";
+        TopicName topicName = TopicName.get(topic);
+        pulsarClient.createProducer(topic);
         pulsarClient.close();
 
-        assertTrue(pulsar.getBrokerService().getTopicReference(topicName) != null);
+        assertTrue(pulsar.getBrokerService().getTopicReference(topic) != null);
         assertTrue(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers()
-                .containsKey(destinationName.getPersistenceNamingEncoding()));
+                .containsKey(topicName.getPersistenceNamingEncoding()));
 
         admin.namespaces().unload("prop/use/ns-abc");
 
         int i = 0;
         for (i = 0; i < 30; i++) {
-            if (pulsar.getBrokerService().getTopicReference(topicName) == null) {
+            if (pulsar.getBrokerService().getTopicReference(topic) == null) {
                 break;
             }
             Thread.sleep(1000);
@@ -572,7 +572,7 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
 
         // ML should have been closed as well
         assertFalse(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers()
-                .containsKey(destinationName.getPersistenceNamingEncoding()));
+                .containsKey(topicName.getPersistenceNamingEncoding()));
     }
 
     @Test
@@ -1128,7 +1128,7 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
         // sleep 1 sec to caclulate metrics per second
         Thread.sleep(1000);
         brokerService.updateRates();
-        List<Metrics> metrics = brokerService.getDestinationMetrics();
+        List<Metrics> metrics = brokerService.getTopicMetrics();
         for (int i = 0; i < metrics.size(); i++) {
             if (metrics.get(i).getDimension("namespace").equalsIgnoreCase(namespace)) {
                 metric = metrics.get(i);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
index b088587..b19b053 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
@@ -93,7 +93,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
@@ -170,7 +170,7 @@ public class PersistentTopicTest {
         NamespaceService nsSvc = mock(NamespaceService.class);
         doReturn(nsSvc).when(pulsar).getNamespaceService();
         doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
-        doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+        doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
 
         setupMLAsyncCallbackMocks();
     }
@@ -1117,7 +1117,7 @@ public class PersistentTopicTest {
         // step-2 now, policies doesn't have removed replication cluster so, it should not invoke "startProducer" of the
         // replicator
         when(pulsar.getConfigurationCache().policiesCache()
-                .get(AdminResource.path(POLICIES, DestinationName.get(globalTopicName).getNamespace())))
+                .get(AdminResource.path(POLICIES, TopicName.get(globalTopicName).getNamespace())))
                         .thenReturn(Optional.of(new Policies()));
         // try to start replicator again
         topic.startReplProducers();
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 5fd74a9..1964db9 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
@@ -63,7 +63,7 @@ import org.apache.pulsar.client.impl.ProducerImpl;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
 import org.apache.pulsar.common.api.Commands;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
@@ -124,10 +124,10 @@ public class ReplicatorTest extends ReplicatorTestBase {
         log.info("--- Starting ReplicatorTest::testConfigChange ---");
         // This test is to verify that the config change on global namespace is successfully applied in broker during
         // runtime.
-        // Run a set of producer tasks to create the destinations
+        // Run a set of producer tasks to create the topics
         List<Future<Void>> results = Lists.newArrayList();
         for (int i = 0; i < 10; i++) {
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/topic-%d", i));
 
             results.add(executor.submit(new Callable<Void>() {
@@ -212,7 +212,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
         final String namespace = "pulsar/global/concurrent";
         admin1.namespaces().createNamespace(namespace);
         admin1.namespaces().setNamespaceReplicationClusters(namespace, Lists.newArrayList("r1", "r2"));
-        final DestinationName dest = DestinationName.get(String.format("persistent://" + namespace + "/topic-%d", 0));
+        final TopicName dest = TopicName.get(String.format("persistent://" + namespace + "/topic-%d", 0));
         ClientConfiguration conf = new ClientConfiguration();
         conf.setStatsInterval(0, TimeUnit.SECONDS);
         Producer producer = PulsarClient.create(url1.toString(), conf).createProducer(dest.toString());
@@ -288,11 +288,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
 
         // This test is to verify that the config change on global namespace is successfully applied in broker during
         // runtime.
-        // Run a set of producer tasks to create the destinations
+        // Run a set of producer tasks to create the topics
         SortedSet<String> testDests = new TreeSet<String>();
         List<Future<Void>> results = Lists.newArrayList();
         for (int i = 0; i < 3; i++) {
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/repltopic-%d", i));
             testDests.add(dest.toString());
 
@@ -389,11 +389,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
 
         // This test is to verify that the config change on global namespace is successfully applied in broker during
         // runtime.
-        // Run a set of producer tasks to create the destinations
+        // Run a set of producer tasks to create the topics
         SortedSet<String> testDests = new TreeSet<String>();
         List<Future<Void>> results = Lists.newArrayList();
         for (int i = 0; i < 10; i++) {
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/repltopic-%d", i));
             testDests.add(dest.toString());
 
@@ -473,7 +473,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
         try {
             // 1. Create a consumer using the reserved consumer id prefix "pulsar.repl."
 
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/res-cons-id"));
 
             // Create another consumer using replication prefix as sub id
@@ -491,7 +491,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
 
         SortedSet<String> testDests = new TreeSet<String>();
 
-        final DestinationName dest = DestinationName.get("persistent://pulsar/global/ns/peekAndSeekTopic");
+        final TopicName dest = TopicName.get("persistent://pulsar/global/ns/peekAndSeekTopic");
         testDests.add(dest.toString());
 
         MessageProducer producer1 = new MessageProducer(url1, dest);
@@ -515,7 +515,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
         // This test is to verify that reset cursor fails on global topic
         SortedSet<String> testDests = new TreeSet<String>();
 
-        final DestinationName dest = DestinationName.get("persistent://pulsar/global/ns/clearBacklogTopic");
+        final TopicName dest = TopicName.get("persistent://pulsar/global/ns/clearBacklogTopic");
         testDests.add(dest.toString());
 
         MessageProducer producer1 = new MessageProducer(url1, dest);
@@ -545,7 +545,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
         SortedSet<String> testDests = new TreeSet<String>();
         List<Future<Void>> results = Lists.newArrayList();
         for (int i = 0; i < 1; i++) {
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/resetrepltopic-%d", i));
             testDests.add(dest.toString());
 
@@ -587,11 +587,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
 
         log.info("--- Starting ReplicatorTest::testReplicationForBatchMessages ---");
 
-        // Run a set of producer tasks to create the destinations
+        // Run a set of producer tasks to create the topics
         SortedSet<String> testDests = new TreeSet<String>();
         List<Future<Void>> results = Lists.newArrayList();
         for (int i = 0; i < 3; i++) {
-            final DestinationName dest = DestinationName
+            final TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns/repltopicbatch-%d", i));
             testDests.add(dest.toString());
 
@@ -660,7 +660,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
     public void testDeleteReplicatorFailure() throws Exception {
         log.info("--- Starting ReplicatorTest::testDeleteReplicatorFailure ---");
         final String topicName = "persistent://pulsar/global/ns/repltopicbatch";
-        final DestinationName dest = DestinationName.get(topicName);
+        final TopicName dest = TopicName.get(topicName);
         MessageProducer producer1 = new MessageProducer(url1, dest);
         PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName);
         final String replicatorClusterName = topic.getReplicators().keys().get(0);
@@ -699,7 +699,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
     public void testReplicatorProducerClosing() throws Exception {
         log.info("--- Starting ReplicatorTest::testDeleteReplicatorFailure ---");
         final String topicName = "persistent://pulsar/global/ns/repltopicbatch";
-        final DestinationName dest = DestinationName.get(topicName);
+        final TopicName dest = TopicName.get(topicName);
         MessageProducer producer1 = new MessageProducer(url1, dest);
         PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName);
         final String replicatorClusterName = topic.getReplicators().keys().get(0);
@@ -735,7 +735,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
             admin1.namespaces().setBacklogQuota("pulsar/global/ns1", new BacklogQuota(1 * 1024 * 1024, policy));
             Thread.sleep(200);
 
-            DestinationName dest = DestinationName
+            TopicName dest = TopicName
                     .get(String.format("persistent://pulsar/global/ns1/%s-%d", policy, System.currentTimeMillis()));
 
             // Producer on r1
@@ -796,7 +796,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
     @Test(timeOut = 5000)
     public void testCloseReplicatorStartProducer() throws Exception {
 
-        DestinationName dest = DestinationName.get("persistent://pulsar/global/ns1/closeCursor");
+        TopicName dest = TopicName.get("persistent://pulsar/global/ns1/closeCursor");
         // Producer on r1
         MessageProducer producer1 = new MessageProducer(url1, dest);
         // Consumer on r1
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java
index 4320428..d71f484 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java
@@ -43,7 +43,7 @@ import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.Producer;
 import org.apache.pulsar.client.api.ProducerConfiguration;
 import org.apache.pulsar.client.api.PulsarClient;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
 import org.apache.pulsar.common.util.FutureUtil;
@@ -95,7 +95,7 @@ public class ReplicatorTestBase {
         return 60;
     }
 
-    public boolean isBrokerServicePurgeInactiveDestination() {
+    public boolean isBrokerServicePurgeInactiveTopic() {
         return false;
     }
 
@@ -122,7 +122,7 @@ public class ReplicatorTestBase {
         config1.setWebServicePortTls(webServicePortTls1);
         config1.setZookeeperServers("127.0.0.1:" + zkPort1);
         config1.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-        config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+        config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
         config1.setBrokerServicePurgeInactiveFrequencyInSeconds(
                 inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
         config1.setBrokerServicePort(PortManager.nextFreePort());
@@ -156,7 +156,7 @@ public class ReplicatorTestBase {
         config2.setWebServicePortTls(webServicePortTls2);
         config2.setZookeeperServers("127.0.0.1:" + zkPort2);
         config2.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-        config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+        config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
         config2.setBrokerServicePurgeInactiveFrequencyInSeconds(
                 inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
         config2.setBrokerServicePort(PortManager.nextFreePort());
@@ -190,7 +190,7 @@ public class ReplicatorTestBase {
         config3.setWebServicePortTls(webServicePortTls3);
         config3.setZookeeperServers("127.0.0.1:" + zkPort3);
         config3.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-        config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+        config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
         config3.setBrokerServicePurgeInactiveFrequencyInSeconds(
                 inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
         config3.setBrokerServicePort(PortManager.nextFreePort());
@@ -265,7 +265,7 @@ public class ReplicatorTestBase {
         PulsarClient client;
         Producer producer;
 
-        MessageProducer(URL url, final DestinationName dest) throws Exception {
+        MessageProducer(URL url, final TopicName dest) throws Exception {
             this.url = url;
             this.namespace = dest.getNamespace();
             this.topicName = dest.toString();
@@ -276,7 +276,7 @@ public class ReplicatorTestBase {
 
         }
 
-        MessageProducer(URL url, final DestinationName dest, boolean batch) throws Exception {
+        MessageProducer(URL url, final TopicName dest, boolean batch) throws Exception {
             this.url = url;
             this.namespace = dest.getNamespace();
             this.topicName = dest.toString();
@@ -337,11 +337,11 @@ public class ReplicatorTestBase {
         final PulsarClient client;
         final Consumer consumer;
 
-        MessageConsumer(URL url, final DestinationName dest) throws Exception {
+        MessageConsumer(URL url, final TopicName dest) throws Exception {
             this(url, dest, "sub-id");
         }
 
-        MessageConsumer(URL url, final DestinationName dest, String subId) throws Exception {
+        MessageConsumer(URL url, final TopicName dest, String subId) throws Exception {
             this.url = url;
             this.namespace = dest.getNamespace();
             this.topicName = dest.toString();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
index 6c82e68..486f1cd 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
@@ -89,7 +89,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
 import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
 import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.Policies;
@@ -176,7 +176,7 @@ public class ServerCnxTest {
         namespaceService = mock(NamespaceService.class);
         doReturn(namespaceService).when(pulsar).getNamespaceService();
         doReturn(true).when(namespaceService).isServiceUnitOwned(any(NamespaceBundle.class));
-        doReturn(true).when(namespaceService).isServiceUnitActive(any(DestinationName.class));
+        doReturn(true).when(namespaceService).isServiceUnitActive(any(TopicName.class));
 
         setupMLAsyncCallbackMocks();
 
@@ -411,7 +411,7 @@ public class ServerCnxTest {
         setChannelConnected();
 
         // Force the case where the broker doesn't own any topic
-        doReturn(false).when(namespaceService).isServiceUnitActive(any(DestinationName.class));
+        doReturn(false).when(namespaceService).isServiceUnitActive(any(TopicName.class));
 
         // test PRODUCER failure case
         ByteBuf clientCommand = Commands.newProducer(nonOwnedTopicName, 1 /* producer id */, 1 /* request id */,
@@ -502,7 +502,7 @@ public class ServerCnxTest {
         doReturn(authorizationService).when(brokerService).getAuthorizationService();
         doReturn(true).when(brokerService).isAuthorizationEnabled();
         doReturn(false).when(authorizationProvider).isSuperUser(Mockito.anyString());
-        doReturn(CompletableFuture.completedFuture(true)).when(authorizationProvider).checkPermission(any(DestinationName.class), Mockito.anyString(),
+        doReturn(CompletableFuture.completedFuture(true)).when(authorizationProvider).checkPermission(any(TopicName.class), Mockito.anyString(),
                 any(AuthAction.class));
 
         resetChannel();
@@ -1238,8 +1238,8 @@ public class ServerCnxTest {
         Policies policies = mock(Policies.class);
         policies.encryption_required = true;
         policies.clusterDispatchRate = Maps.newHashMap();
-        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
-        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
         doReturn(zkDataCache).when(configCacheService).policiesCache();
 
         // test success case: encrypted producer can connect
@@ -1266,8 +1266,8 @@ public class ServerCnxTest {
         Policies policies = mock(Policies.class);
         policies.encryption_required = true;
         policies.clusterDispatchRate = Maps.newHashMap();
-        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
-        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
         doReturn(zkDataCache).when(configCacheService).policiesCache();
 
         // test failure case: unencrypted producer cannot connect
@@ -1296,8 +1296,8 @@ public class ServerCnxTest {
         Policies policies = mock(Policies.class);
         policies.encryption_required = true;
         policies.clusterDispatchRate = Maps.newHashMap();
-        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
-        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
         doReturn(zkDataCache).when(configCacheService).policiesCache();
 
         ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */,
@@ -1331,8 +1331,8 @@ public class ServerCnxTest {
         Policies policies = mock(Policies.class);
         policies.encryption_required = true;
         policies.clusterDispatchRate = Maps.newHashMap();
-        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
-        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+        doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
         doReturn(zkDataCache).when(configCacheService).policiesCache();
 
         ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */,
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
index e950fbd..5288640 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
@@ -227,7 +227,7 @@ public class ZooKeeperClientAspectJTest {
 
             BrokerService brokerService = pulsar.getBrokerService();
             brokerService.updateRates();
-            List<Metrics> metrics = brokerService.getDestinationMetrics();
+            List<Metrics> metrics = brokerService.getTopicMetrics();
             AtomicDouble writeRate = new AtomicDouble();
             AtomicDouble readRate = new AtomicDouble();
             metrics.forEach(m -> {
@@ -250,7 +250,7 @@ public class ZooKeeperClientAspectJTest {
     private Metrics getMetric(PulsarService pulsar, String dimension) {
         BrokerService brokerService = pulsar.getBrokerService();
         brokerService.updateRates();
-        for (Metrics metric : brokerService.getDestinationMetrics()) {
+        for (Metrics metric : brokerService.getTopicMetrics()) {
             if (dimension.equalsIgnoreCase(metric.getDimension("metric"))) {
                 return metric;
             }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
index 7292940..7c6f6a9 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
@@ -307,17 +307,17 @@ public class AuthenticatedProducerConsumerTest extends ProducerConsumerBase {
         String namespace = "my-property/use/my-ns";
         admin.namespaces().createNamespace(namespace);
 
-        String destination = "persistent://" + namespace + "1/topic1";
+        String topic = "persistent://" + namespace + "1/topic1";
         // this will cause NPE and it should throw 500
         mockZookKeeper.shutdown();
         pulsar.getConfiguration().setSuperUserRoles(Sets.newHashSet());
         try {
-            admin.persistentTopics().getPartitionedTopicMetadata(destination);
+            admin.persistentTopics().getPartitionedTopicMetadata(topic);
         } catch (PulsarAdminException e) {
             Assert.assertTrue(e.getCause() instanceof InternalServerErrorException);
         }
         try {
-            admin.lookups().lookupDestination(destination);
+            admin.lookups().lookupTopic(topic);
         } catch (PulsarAdminException e) {
             Assert.assertTrue(e.getCause() instanceof InternalServerErrorException);
         }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
index 465cc6d..1eb27db 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.authorization.AuthorizationProvider;
 import org.apache.pulsar.broker.authorization.AuthorizationService;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
 import org.apache.pulsar.client.admin.PulsarAdmin;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -86,13 +86,13 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
 
     /**
      * It verifies plugable authorization service
-     * 
+     *
      * <pre>
      * 1. Client passes correct authorization plugin-name + correct auth role: SUCCESS
      * 2. Client passes correct authorization plugin-name + incorrect auth-role: FAIL
      * 3. Client passes incorrect authorization plugin-name + correct auth-role: FAIL
      * </pre>
-     * 
+     *
      * @throws Exception
      */
     @Test
@@ -197,14 +197,14 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         setup();
 
         AuthorizationService authorizationService = new AuthorizationService(conf, null);
-        DestinationName destination = DestinationName.get("persistent://prop/cluster/ns/t1");
+        TopicName topicName = TopicName.get("persistent://prop/cluster/ns/t1");
         String role = "test-role";
-        Assert.assertFalse(authorizationService.canProduce(destination, role, null));
-        Assert.assertFalse(authorizationService.canConsume(destination, role, null, "sub1"));
+        Assert.assertFalse(authorizationService.canProduce(topicName, role, null));
+        Assert.assertFalse(authorizationService.canConsume(topicName, role, null, "sub1"));
         authorizationService
-                .grantPermissionAsync(destination, null, role, "auth-json").get();
-        Assert.assertTrue(authorizationService.canProduce(destination, role, null));
-        Assert.assertTrue(authorizationService.canConsume(destination, role, null, "sub1"));
+                .grantPermissionAsync(topicName, null, role, "auth-json").get();
+        Assert.assertTrue(authorizationService.canProduce(topicName, role, null));
+        Assert.assertTrue(authorizationService.canConsume(topicName, role, null, "sub1"));
 
         log.info("-- Exiting {} test --", methodName);
     }
@@ -217,17 +217,17 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         setup();
 
         AuthorizationService authorizationService = new AuthorizationService(conf, null);
-        DestinationName destination = DestinationName.get("persistent://prop/cluster/ns/t1");
+        TopicName topicName = TopicName.get("persistent://prop/cluster/ns/t1");
         String role = "test-role";
         authorizationService
-                .grantPermissionAsync(destination, null, role, "auth-json")
+                .grantPermissionAsync(topicName, null, role, "auth-json")
                 .get();
         Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authDataJson, "auth-json");
         Assert.assertTrue(
-                authorizationService.canProduce(destination, role, new AuthenticationDataCommand("prod-auth")));
+                authorizationService.canProduce(topicName, role, new AuthenticationDataCommand("prod-auth")));
         Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authenticationData.getCommandData(),
                 "prod-auth");
-        Assert.assertTrue(authorizationService.canConsume(destination, role, new AuthenticationDataCommand("cons-auth"),
+        Assert.assertTrue(authorizationService.canConsume(topicName, role, new AuthenticationDataCommand("cons-auth"),
                 "sub1"));
         Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authenticationData.getCommandData(),
                 "cons-auth");
@@ -324,19 +324,19 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         }
 
         @Override
-        public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             return CompletableFuture.completedFuture(clientRole.equals(role));
         }
 
         @Override
-        public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData, String subscription) {
             return CompletableFuture.completedFuture(clientRole.equals(role));
         }
 
         @Override
-        public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             return CompletableFuture.completedFuture(clientRole.equals(role));
         }
@@ -348,7 +348,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         }
 
         @Override
-        public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions,
+        public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions,
                 String role, String authenticationData) {
             return CompletableFuture.completedFuture(null);
         }
@@ -361,19 +361,19 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
     public static class TestAuthorizationProvider2 extends TestAuthorizationProvider {
 
         @Override
-        public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             return CompletableFuture.completedFuture(true);
         }
 
         @Override
-        public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData, String subscription) {
             return CompletableFuture.completedFuture(false);
         }
 
         @Override
-        public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             return CompletableFuture.completedFuture(true);
         }
@@ -382,7 +382,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
     public static class TestAuthorizationProviderWithSubscriptionPrefix extends TestAuthorizationProvider {
 
         @Override
-        public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData, String subscription) {
             CompletableFuture<Boolean> future = new CompletableFuture<>();
             if (isNotBlank(subscription)) {
@@ -404,21 +404,21 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         static String authDataJson;
 
         @Override
-        public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             this.authenticationData = authenticationData;
             return CompletableFuture.completedFuture(grantRoles.contains(role));
         }
 
         @Override
-        public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData, String subscription) {
             this.authenticationData = authenticationData;
             return CompletableFuture.completedFuture(grantRoles.contains(role));
         }
 
         @Override
-        public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+        public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
                 AuthenticationDataSource authenticationData) {
             this.authenticationData = authenticationData;
             return CompletableFuture.completedFuture(grantRoles.contains(role));
@@ -433,7 +433,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
         }
 
         @Override
-        public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions,
+        public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions,
                 String role, String authData) {
             this.authDataJson = authData;
             grantRoles.add(role);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
index 80d2d00..c72aa9e 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
@@ -72,7 +72,7 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit;
 import org.apache.pulsar.broker.namespace.NamespaceService;
 import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
 import org.apache.pulsar.client.impl.auth.AuthenticationTls;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.ServiceUnitId;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
@@ -310,12 +310,12 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         int numPartitions = 8;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setSubscriptionType(SubscriptionType.Exclusive);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         /**** start broker-2 ****/
         ServiceConfiguration conf2 = new ServiceConfiguration();
@@ -348,9 +348,9 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         for (int i = 0; i < 20; i++) {
             String message = "my-message-" + i;
@@ -371,7 +371,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         pulsar2.close();
         loadManager2 = null;
@@ -846,8 +846,8 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         final String unsplitBundle = namespace + "/0x00000000_0xffffffff";
         assertTrue(serviceUnits1.contains(unsplitBundle));
         // broker-2 should have this bundle into the cache
-        DestinationName destination = DestinationName.get(topic1);
-        NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(destination);
+        TopicName topicName = TopicName.get(topic1);
+        NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(topicName);
         assertEquals(bundleInBroker2.toString(), unsplitBundle);
 
         // (5) Split the bundle for topic-1
@@ -856,7 +856,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         // (6) Broker-2 should get the watch and update bundle cache
         final int retry = 5;
         for (int i = 0; i < retry; i++) {
-            if (pulsar2.getNamespaceService().getBundle(destination).equals(bundleInBroker2) && i != retry - 1) {
+            if (pulsar2.getNamespaceService().getBundle(topicName).equals(bundleInBroker2) && i != retry - 1) {
                 Thread.sleep(200);
             } else {
                 break;
@@ -868,7 +868,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         Consumer consumer2 = pulsarClient.subscribe(topic2, "my-subscriber-name", new ConsumerConfiguration());
 
         NamespaceBundle bundleInBroker1AfterSplit = pulsar2.getNamespaceService()
-                .getBundle(DestinationName.get(topic2));
+                .getBundle(TopicName.get(topic2));
         assertFalse(bundleInBroker1AfterSplit.equals(unsplitBundle));
 
         consumer1.close();
@@ -955,8 +955,8 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
             final String unsplitBundle = namespace + "/0x00000000_0xffffffff";
             assertTrue(serviceUnits1.contains(unsplitBundle));
             // broker-2 should have this bundle into the cache
-            DestinationName destination = DestinationName.get(topic1);
-            NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(destination);
+            TopicName topicName = TopicName.get(topic1);
+            NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(topicName);
             assertEquals(bundleInBroker2.toString(), unsplitBundle);
 
             // update broker-1 bundle report to zk
@@ -985,7 +985,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
             // (6) Broker-2 should get the watch and update bundle cache
             final int retry = 5;
             for (int i = 0; i < retry; i++) {
-                if (pulsar2.getNamespaceService().getBundle(destination).equals(bundleInBroker2) && i != retry - 1) {
+                if (pulsar2.getNamespaceService().getBundle(topicName).equals(bundleInBroker2) && i != retry - 1) {
                     Thread.sleep(200);
                 } else {
                     break;
@@ -997,7 +997,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
             Consumer consumer2 = pulsarClient.subscribe(topic2, "my-subscriber-name", new ConsumerConfiguration());
 
             NamespaceBundle bundleInBroker1AfterSplit = pulsar2.getNamespaceService()
-                    .getBundle(DestinationName.get(topic2));
+                    .getBundle(TopicName.get(topic2));
             assertFalse(bundleInBroker1AfterSplit.equals(unsplitBundle));
 
             consumer1.close();
@@ -1018,7 +1018,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
         final String namespace = "my-ns";
         final String topicName = "my-partitioned";
         final int totalPartitions = 10;
-        final DestinationName dest = DestinationName.get("persistent", property, cluster, namespace, topicName);
+        final TopicName dest = TopicName.get("persistent", property, cluster, namespace, topicName);
         admin.clusters().createCluster(cluster,
                 new ClusterData("http://127.0.0.1:" + BROKER_WEBSERVICE_PORT, null, null, null));
         admin.properties().createProperty(property,
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
index 5f45bc4..fefe8e7 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
@@ -51,7 +51,7 @@ import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.impl.ConsumerImpl;
 import org.apache.pulsar.client.impl.ProducerImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
@@ -203,7 +203,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
 
         // Ensure all partitions exist
         for (int i = 0; i < numPartitions; i++) {
-            DestinationName partition = DestinationName.get(topic).getPartition(i);
+            TopicName partition = TopicName.get(topic).getPartition(i);
             assertNotNull(pulsar.getBrokerService().getTopicReference(partition.toString()));
         }
 
@@ -494,7 +494,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             NonPersistentTopicStats stats;
             SubscriptionStats subStats;
 
-            DestinationName dest = DestinationName.get(globalTopicName);
+            TopicName dest = TopicName.get(globalTopicName);
 
             PulsarClient client1 = PulsarClient.create(replication.url1.toString(), new ClientConfiguration());
             PulsarClient client2 = PulsarClient.create(replication.url2.toString(), new ClientConfiguration());
@@ -652,7 +652,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             manager.start();
             loadManagerRef.set(manager);
 
-            NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+            NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
             LoadManager loadManager = pulsar.getLoadManager().get();
             ResourceUnit broker = null;
             try {
@@ -741,7 +741,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             manager.start();
             loadManagerRef.set(manager);
 
-            NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+            NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
             LoadManager loadManager = pulsar.getLoadManager().get();
             ResourceUnit broker = null;
             try {
@@ -865,7 +865,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             return 60;
         }
 
-        public boolean isBrokerServicePurgeInactiveDestination() {
+        public boolean isBrokerServicePurgeInactiveTopic() {
             return false;
         }
 
@@ -891,7 +891,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             config1.setWebServicePort(webServicePort1);
             config1.setZookeeperServers("127.0.0.1:" + zkPort1);
             config1.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-            config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+            config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
             config1.setBrokerServicePurgeInactiveFrequencyInSeconds(
                     inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
             config1.setBrokerServicePort(PortManager.nextFreePort());
@@ -917,7 +917,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             config2.setAdvertisedAddress("localhost");
             config2.setZookeeperServers("127.0.0.1:" + zkPort2);
             config2.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-            config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+            config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
             config2.setBrokerServicePurgeInactiveFrequencyInSeconds(
                     inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
             config2.setBrokerServicePort(PortManager.nextFreePort());
@@ -943,7 +943,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
             config3.setAdvertisedAddress("localhost");
             config3.setZookeeperServers("127.0.0.1:" + zkPort3);
             config3.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
-            config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+            config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
             config3.setBrokerServicePurgeInactiveFrequencyInSeconds(
                     inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
             config3.setBrokerServicePort(PortManager.nextFreePort());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
index dbe454d..3be9edc 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
@@ -40,7 +40,7 @@ import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
 import org.apache.pulsar.client.impl.PartitionedProducerImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -78,18 +78,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setSubscriptionType(SubscriptionType.Exclusive);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         for (int i = 0; i < 10; i++) {
             String message = "my-message-" + i;
@@ -110,7 +110,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         log.info("-- Exiting {} test --", methodName);
     }
@@ -121,16 +121,15 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
 
         int numPartitions = 4;
         final String specialCharacter = "! * ' ( ) ; : @ & = + $ , /\\ ? % # [ ]";
-        final String topicName = "my-partitionedtopic1" + specialCharacter;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/" + topicName);
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1" + specialCharacter);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
         // Try to create producer which does lookup and create connection with broker
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
         producer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
         log.info("-- Exiting {} test --", methodName);
     }
 
@@ -139,18 +138,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic2");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic2");
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setSubscriptionType(SubscriptionType.Exclusive);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.SinglePartition);
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         for (int i = 0; i < 10; i++) {
             String message = "my-message-" + i;
@@ -173,7 +172,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         log.info("-- Exiting {} test --", methodName);
     }
@@ -183,17 +182,17 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic3");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic3");
         String dummyKey1 = "dummykey1";
         String dummyKey2 = "dummykey2";
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setSubscriptionType(SubscriptionType.Exclusive);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
-        Producer producer = pulsarClient.createProducer(dn.toString());
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Producer producer = pulsarClient.createProducer(topicName.toString());
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         Message msg = null;
         for (int i = 0; i < 5; i++) {
@@ -221,7 +220,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         log.info("-- Exiting {} test --", methodName);
     }
@@ -240,13 +239,13 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic4");
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic4");
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ConsumerConfiguration consumerConf = new ConsumerConfiguration();
         consumerConf.setSubscriptionType(SubscriptionType.Exclusive);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-subscriber-name", consumerConf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-subscriber-name", consumerConf);
 
         try {
             Message msg = MessageBuilder.create().setContent("InvalidMessage".getBytes()).build();
@@ -271,7 +270,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
             // ok
         }
 
-        Producer producer = pulsarClient.createProducer(dn.toString());
+        Producer producer = pulsarClient.createProducer(topicName.toString());
         producer.close();
 
         try {
@@ -281,7 +280,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
             // ok
         }
 
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
     }
 
@@ -289,8 +288,8 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
     public void testSillyUser() throws Exception {
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic5");
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic5");
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         Producer producer = null;
@@ -311,7 +310,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         }
 
         try {
-            producer = pulsarClient.createProducer(dn.toString(), null);
+            producer = pulsarClient.createProducer(topicName.toString(), null);
             Assert.fail("should fail");
         } catch (PulsarClientException e) {
             Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException);
@@ -320,15 +319,15 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         ConsumerConfiguration consumerConf = new ConsumerConfiguration();
 
         try {
-            consumer = pulsarClient.subscribe(dn.toString(), "my-subscriber-name", null);
+            consumer = pulsarClient.subscribe(topicName.toString(), "my-subscriber-name", null);
             Assert.fail("Should fail");
         } catch (PulsarClientException e) {
             Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException);
         }
 
         try {
-            producer = pulsarClient.createProducer(dn.toString());
-            consumer = pulsarClient.subscribe(dn.toString(), "my-sub");
+            producer = pulsarClient.createProducer(topicName.toString());
+            consumer = pulsarClient.subscribe(topicName.toString(), "my-sub");
             producer.send("message1".getBytes());
             producer.send("message2".getBytes());
             Message msg1 = consumer.receive();
@@ -343,25 +342,25 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
             consumer.close();
         }
 
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
     }
 
     @Test(timeOut = 30000)
     public void testDeletePartitionedTopic() throws Exception {
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic6");
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic6");
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
-        Producer producer = pulsarClient.createProducer(dn.toString());
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-sub");
+        Producer producer = pulsarClient.createProducer(topicName.toString());
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-sub");
         consumer.unsubscribe();
         consumer.close();
         producer.close();
 
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
-        Producer producer1 = pulsarClient.createProducer(dn.toString());
+        Producer producer1 = pulsarClient.createProducer(topicName.toString());
         if (producer1 instanceof PartitionedProducerImpl) {
             Assert.fail("should fail since partitioned topic was deleted");
         }
@@ -376,18 +375,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         final Set<String> consumeMsgs = Sets.newHashSet();
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setSubscriptionType(SubscriptionType.Shared);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         // produce messages
         for (int i = 0; i < totalMsg; i++) {
@@ -413,7 +412,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         log.info("-- Exiting {} test --", methodName);
     }
@@ -427,18 +426,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         final Set<String> consumeMsgs = Sets.newHashSet();
 
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
 
         ConsumerConfiguration conf = new ConsumerConfiguration();
         conf.setReceiverQueueSize(1);
 
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
 
         ProducerConfiguration producerConf = new ProducerConfiguration();
         producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
-        Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+        Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
 
-        Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+        Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
 
         // produce messages
         for (int i = 0; i < totalMsg; i++) {
@@ -464,7 +463,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
         producer.close();
         consumer.unsubscribe();
         consumer.close();
-        admin.persistentTopics().deletePartitionedTopic(dn.toString());
+        admin.persistentTopics().deletePartitionedTopic(topicName.toString());
 
         log.info("-- Exiting {} test --", methodName);
     }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
index 9635c43..33d1063 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.client.impl.ConsumerImpl;
 import org.apache.pulsar.client.impl.MessageIdImpl;
 import org.apache.pulsar.common.api.PulsarDecoder;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.util.FutureUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -2152,9 +2152,9 @@ public class SimpleProducerConsumerTest extends ProducerConsumerBase {
 
         // (2) Partitioned-consumer
         int numPartitions = 4;
-        DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/failAsyncReceive");
-        admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
-        Consumer partitionedConsumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber",
+        TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/failAsyncReceive");
+        admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
+        Consumer partitionedConsumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber",
                 new ConsumerConfiguration());
         partitionedConsumer.close();
         // receive messages
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
index 5454bb7..8cf5618 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
@@ -64,7 +64,7 @@ import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.impl.HandlerBase.State;
 import org.apache.pulsar.common.api.PulsarHandler;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
@@ -126,12 +126,12 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
         admin.namespaces().createNamespace(ns1);
         admin.namespaces().createNamespace(ns2);
 
-        final String dn1 = "persistent://" + ns1 + "/my-topic";
-        final String dn2 = "persistent://" + ns2 + "/my-topic";
-        ConsumerImpl cons1 = (ConsumerImpl) pulsarClient.subscribe(dn1, "my-subscriber-name",
+        final String topic1 = "persistent://" + ns1 + "/my-topic";
+        final String topic2 = "persistent://" + ns2 + "/my-topic";
+        ConsumerImpl cons1 = (ConsumerImpl) pulsarClient.subscribe(topic1, "my-subscriber-name",
                 new ConsumerConfiguration());
-        ProducerImpl prod1 = (ProducerImpl) pulsarClient.createProducer(dn1, new ProducerConfiguration());
-        ProducerImpl prod2 = (ProducerImpl) pulsarClient.createProducer(dn2, new ProducerConfiguration());
+        ProducerImpl prod1 = (ProducerImpl) pulsarClient.createProducer(topic1, new ProducerConfiguration());
+        ProducerImpl prod2 = (ProducerImpl) pulsarClient.createProducer(topic2, new ProducerConfiguration());
         ConsumerImpl consumer1 = spy(cons1);
         doAnswer(invocationOnMock -> cons1.getState()).when(consumer1).getState();
         doAnswer(invocationOnMock -> cons1.getClientCnx()).when(consumer1).getClientCnx();
@@ -178,8 +178,8 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
         // disable this broker to avoid any new requests
         pulsar.getLoadManager().get().disableBroker();
 
-        NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn1));
-        NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn2));
+        NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(TopicName.get(topic1));
+        NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(TopicName.get(topic2));
 
         // unload ns-bundle:1
         pulsar.getNamespaceService().unloadNamespaceBundle((NamespaceBundle) bundle1);
@@ -245,13 +245,13 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
         admin.namespaces().createNamespace(ns1);
         admin.namespaces().createNamespace(ns2);
 
-        final String dn1 = "persistent://" + ns1 + "/my-topic";
-        final String dn2 = "persistent://" + ns2 + "/my-topic";
+        final String topic1 = "persistent://" + ns1 + "/my-topic";
+        final String topic2 = "persistent://" + ns2 + "/my-topic";
 
-        ConsumerImpl consumer1 = (ConsumerImpl) pulsarClient.subscribe(dn1, "my-subscriber-name",
+        ConsumerImpl consumer1 = (ConsumerImpl) pulsarClient.subscribe(topic1, "my-subscriber-name",
                 new ConsumerConfiguration());
-        ProducerImpl producer1 = (ProducerImpl) pulsarClient.createProducer(dn1, new ProducerConfiguration());
-        ProducerImpl producer2 = (ProducerImpl) pulsarClient.createProducer(dn2, new ProducerConfiguration());
+        ProducerImpl producer1 = (ProducerImpl) pulsarClient.createProducer(topic1, new ProducerConfiguration());
+        ProducerImpl producer2 = (ProducerImpl) pulsarClient.createProducer(topic2, new ProducerConfiguration());
 
         // unload all other namespace
         pulsar.getBrokerService().close();
@@ -376,7 +376,7 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
     @Test(timeOut = 10000, dataProvider = "subType")
     public void testResetCursor(SubscriptionType subType) throws Exception {
         final RetentionPolicies policy = new RetentionPolicies(60, 52 * 1024);
-        final DestinationName destName = DestinationName.get("persistent://my-property/use/my-ns/unacked-topic");
+        final TopicName destName = TopicName.get("persistent://my-property/use/my-ns/unacked-topic");
         final int warmup = 20;
         final int testSize = 150;
         final List<Message> received = new ArrayList<Message>();
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 e86f516..a577741 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
@@ -29,7 +29,7 @@ import static org.testng.Assert.fail;
 
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -189,20 +189,20 @@ public class NamespaceBundleTest {
 
     @Test
     public void testIncludes() throws Exception {
-        DestinationName dn = DestinationName.get("persistent://pulsar/use/ns1/topic-1");
-        Long hashKey = factory.getLongHashCode(dn.toString());
+        TopicName topicName = TopicName.get("persistent://pulsar/use/ns1/topic-1");
+        Long hashKey = factory.getLongHashCode(topicName.toString());
         Long upper = Math.max(hashKey + 1, NamespaceBundles.FULL_UPPER_BOUND);
         BoundType upperType = upper.equals(NamespaceBundles.FULL_UPPER_BOUND) ? BoundType.CLOSED : BoundType.OPEN;
-        NamespaceBundle bundle = factory.getBundle(dn.getNamespaceObject(),
+        NamespaceBundle bundle = factory.getBundle(topicName.getNamespaceObject(),
                 Range.range(hashKey / 2, BoundType.CLOSED, upper, upperType));
-        assertTrue(bundle.includes(dn));
+        assertTrue(bundle.includes(topicName));
         bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
                 Range.range(upper, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.CLOSED));
-        assertTrue(!bundle.includes(dn));
+        assertTrue(!bundle.includes(topicName));
 
         NamespaceBundle otherBundle = factory.getBundle(NamespaceName.get("pulsar/use/ns2"),
                 Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
-        assertTrue(!otherBundle.includes(dn));
+        assertTrue(!otherBundle.includes(topicName));
     }
 
     @Test
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 4c2bebe..310a834 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
@@ -37,7 +37,7 @@ import java.util.concurrent.CompletableFuture;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceBundle;
 import org.apache.pulsar.common.naming.NamespaceBundleFactory;
 import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -143,19 +143,19 @@ public class NamespaceBundlesTest {
         partitions.add(0xc0000000l);
         partitions.add(0xffffffffl);
         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));
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
+        NamespaceBundle bundle = bundles.findBundle(topicName);
+        assertTrue(bundle.includes(topicName));
 
-        dn = DestinationName.get("persistent://pulsar/use/ns2/topic-2");
+        topicName = TopicName.get("persistent://pulsar/use/ns2/topic-2");
         try {
-            bundles.findBundle(dn);
+            bundles.findBundle(topicName);
             fail("Should have failed due to mismatched namespace name");
         } catch (IllegalArgumentException iae) {
             // OK, expected
         }
 
-        Long hashKey = factory.getLongHashCode(dn.toString());
+        Long hashKey = factory.getLongHashCode(topicName.toString());
         // The following code guarantees that we have at least two ranges after the hashKey till the end
         SortedSet<Long> tailSet = partitions.tailSet(hashKey);
         tailSet.add(hashKey);
@@ -165,8 +165,8 @@ public class NamespaceBundlesTest {
         SortedSet<Long> newPar = tailSet.tailSet(iter.next());
 
         try {
-            bundles = new NamespaceBundles(dn.getNamespaceObject(), newPar, factory);
-            bundles.findBundle(dn);
+            bundles = new NamespaceBundles(topicName.getNamespaceObject(), newPar, factory);
+            bundles.findBundle(topicName);
             fail("Should have failed due to out-of-range");
         } catch (ArrayIndexOutOfBoundsException iae) {
             // OK, expected
@@ -176,9 +176,9 @@ public class NamespaceBundlesTest {
     @Test
     public void testsplitBundles() throws Exception {
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = factory.getBundles(nsname);
-        NamespaceBundle bundle = bundles.findBundle(dn);
+        NamespaceBundle bundle = bundles.findBundle(topicName);
         final int numberSplitBundles = 4;
         // (1) split in 4
         Pair<NamespaceBundles, List<NamespaceBundle>> splitBundles = factory.splitBundles(bundle, numberSplitBundles);
@@ -220,9 +220,9 @@ public class NamespaceBundlesTest {
     public void testSplitBundleInTwo() throws Exception {
         final int NO_BUNDLES = 2;
         NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
-        DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+        TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
         NamespaceBundles bundles = factory.getBundles(nsname);
-        NamespaceBundle bundle = bundles.findBundle(dn);
+        NamespaceBundle bundle = bundles.findBundle(topicName);
         // (1) split : [0x00000000,0xffffffff] => [0x00000000_0x7fffffff,0x7fffffff_0xffffffff]
         Pair<NamespaceBundles, List<NamespaceBundle>> splitBundles = factory.splitBundles(bundle, NO_BUNDLES);
         assertNotNull(splitBundles);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
index 943c62d..4688e14 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
@@ -29,7 +29,7 @@ import org.apache.bookkeeper.test.PortManager;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.authorization.AuthorizationService;
 import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -80,7 +80,7 @@ public class ProxyAuthorizationTest extends MockedPulsarServiceBaseTest {
     public void test() throws Exception {
         AuthorizationService auth = service.getAuthorizationService();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
 
         admin.clusters().updateCluster(configClusterName, new ClusterData());
         admin.properties().createProperty("p1", new PropertyAdmin(Lists.newArrayList("role1"), Sets.newHashSet("c1")));
@@ -88,31 +88,31 @@ public class ProxyAuthorizationTest extends MockedPulsarServiceBaseTest {
         admin.namespaces().createNamespace("p1/c1/ns1");
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.of(AuthAction.produce));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
 
         admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role",
                 EnumSet.of(AuthAction.consume));
         waitForChange();
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
 
-        assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
+        assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
 
         admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.allOf(AuthAction.class));
         waitForChange();
 
-        assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
-        assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
+        assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+        assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
 
         admin.namespaces().deleteNamespace("p1/c1/ns1");
         admin.properties().deleteProperty("p1");
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
index 6fe07c7..16ebd5e 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
@@ -31,7 +31,7 @@ public interface BrokerStats {
 
     /**
      * Returns Monitoring metrics
-     * 
+     *
      * @return
      * @throws PulsarAdminException
      */
@@ -42,25 +42,25 @@ public interface BrokerStats {
      * Requests JSON string server mbean dump
      * <p>
      * Notes: since we don't plan to introspect the response we avoid converting the response into POJO.
-     * 
+     *
      * @return
      * @throws PulsarAdminException
      */
     JsonArray getMBeans() throws PulsarAdminException;
 
     /**
-     * Returns JSON string destination stats
+     * Returns JSON string topics stats
      * <p>
      * Notes: since we don't plan to introspect the response we avoid converting the response into POJO.
-     * 
+     *
      * @return
      * @throws PulsarAdminException
      */
-    JsonObject getDestinations() throws PulsarAdminException;
+    JsonObject getTopics() throws PulsarAdminException;
 
     JsonObject getPendingBookieOpsStats() throws PulsarAdminException;
 
     AllocatorStats getAllocatorStats(String allocatorName) throws PulsarAdminException;
-    
+
     LoadManagerReport getLoadReport() throws PulsarAdminException;
 }
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
index e578890..10d244b 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
@@ -19,25 +19,24 @@
 package org.apache.pulsar.client.admin;
 
 /**
- * This is an interface class to allow using command line tool to quickly lookup the broker serving the
- * destination.
+ * This is an interface class to allow using command line tool to quickly lookup the broker serving the topic.
  */
 public interface Lookup {
 
     /**
-     * Lookup a destination
+     * Lookup a topic
      *
-     * @param destination
-     * @return the broker URL that serves the destination
+     * @param topic
+     * @return the broker URL that serves the topic
      */
-    public String lookupDestination(String destination) throws PulsarAdminException;
-    
+    public String lookupTopic(String topic) throws PulsarAdminException;
+
     /**
-     * Get a bundle range of a destination
-     * 
-     * @param destination
+     * Get a bundle range of a topic
+     *
+     * @param topic
      * @return
      * @throws PulsarAdminException
      */
-    public String getBundleRange(String destination) throws PulsarAdminException;
+    public String getBundleRange(String topic) throws PulsarAdminException;
 }
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
index 3c8d3d5..65ce200 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
@@ -90,9 +90,9 @@ public interface Namespaces {
     List<String> getNamespaces(String property, String cluster) throws PulsarAdminException;
 
     /**
-     * Get the list of destinations.
+     * Get the list of topics.
      * <p>
-     * Get the list of all the destinations under a certain namespace.
+     * Get the list of all the topics under a certain namespace.
      * <p>
      * Response Example:
      *
@@ -111,7 +111,7 @@ public interface Namespaces {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    List<String> getDestinations(String namespace) throws PulsarAdminException;
+    List<String> getTopics(String namespace) throws PulsarAdminException;
 
     /**
      * Get policies for a namespace.
@@ -420,8 +420,8 @@ public interface Namespaces {
      *             Unexpected error
      */
     void setNamespaceMessageTTL(String namespace, int ttlInSeconds) throws PulsarAdminException;
-    
-    
+
+
     /**
      * Set anti-affinity group name for a namespace
      * <p>
@@ -440,10 +440,10 @@ public interface Namespaces {
      *             Unexpected error
      */
     void setNamespaceAntiAffinityGroup(String namespace, String namespaceAntiAffinityGroup) throws PulsarAdminException;
-    
+
     /**
      * Get all namespaces that grouped with given anti-affinity group
-     * 
+     *
      * @param property
      *            property is only used for authorization. Client has to be admin of any of the property to access this
      *            api api.
@@ -566,7 +566,7 @@ public interface Namespaces {
     Map<BacklogQuota.BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(String namespace) throws PulsarAdminException;
 
     /**
-     * Set a backlog quota for all the destinations on a namespace.
+     * Set a backlog quota for all the topics on a namespace.
      * <p>
      * Set a backlog quota on a namespace.
      * <p>
@@ -617,7 +617,7 @@ public interface Namespaces {
     public void removeBacklogQuota(String namespace) throws PulsarAdminException;
 
     /**
-     * Set the persistence configuration for all the destinations on a namespace.
+     * Set the persistence configuration for all the topics on a namespace.
      * <p>
      * Set the persistence configuration on a namespace.
      * <p>
@@ -695,7 +695,7 @@ public interface Namespaces {
     public PersistencePolicies getPersistence(String namespace) throws PulsarAdminException;
 
     /**
-     * Set the retention configuration for all the destinations on a namespace.
+     * Set the retention configuration for all the topics on a namespace.
      * <p/>
      * Set the retention configuration on a namespace. This operation requires Pulsar super-user access.
      * <p/>
@@ -825,7 +825,7 @@ public interface Namespaces {
     DispatchRate getDispatchRate(String namespace) throws PulsarAdminException;
 
     /**
-     * Clear backlog for all destinations on a namespace
+     * Clear backlog for all topics on a namespace
      *
      * @param namespace
      * @throws PulsarAdminException
@@ -834,7 +834,7 @@ public interface Namespaces {
     void clearNamespaceBacklog(String namespace) throws PulsarAdminException;
 
     /**
-     * Clear backlog for a given subscription on all destinations on a namespace
+     * Clear backlog for a given subscription on all topics on a namespace
      *
      * @param namespace
      * @param subscription
@@ -844,7 +844,7 @@ public interface Namespaces {
     void clearNamespaceBacklogForSubscription(String namespace, String subscription) throws PulsarAdminException;
 
     /**
-     * Clear backlog for all destinations on a namespace bundle
+     * Clear backlog for all topics on a namespace bundle
      *
      * @param namespace
      * @param bundle
@@ -854,7 +854,7 @@ public interface Namespaces {
     void clearNamespaceBundleBacklog(String namespace, String bundle) throws PulsarAdminException;
 
     /**
-     * Clear backlog for a given subscription on all destinations on a namespace bundle
+     * Clear backlog for a given subscription on all topics on a namespace bundle
      *
      * @param namespace
      * @param bundle
@@ -866,7 +866,7 @@ public interface Namespaces {
             throws PulsarAdminException;
 
     /**
-     * Unsubscribes the given subscription on all destinations on a namespace
+     * Unsubscribes the given subscription on all topics on a namespace
      *
      * @param namespace
      * @param subscription
@@ -875,7 +875,7 @@ public interface Namespaces {
     void unsubscribeNamespace(String namespace, String subscription) throws PulsarAdminException;
 
     /**
-     * Unsubscribes the given subscription on all destinations on a namespace bundle
+     * Unsubscribes the given subscription on all topics on a namespace bundle
      *
      * @param namespace
      * @param bundle
@@ -910,7 +910,7 @@ public interface Namespaces {
     void setEncryptionRequiredStatus(String namespace, boolean encryptionRequired) throws PulsarAdminException;
 
      /**
-     * Set the given subscription auth mode on all destinations on a namespace
+     * Set the given subscription auth mode on all topics on a namespace
      *
      * @param namespace
      * @param subscriptionAuthMode
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
index c21ea9a..2bef501 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
@@ -37,12 +37,12 @@ public interface NonPersistentTopics {
      * Get metadata of a partitioned topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return Partitioned topic metadata
      * @throws PulsarAdminException
      */
-    PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException;
+    PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException;
 
     /**
      * Get metadata of a partitioned topic asynchronously.
@@ -50,11 +50,11 @@ public interface NonPersistentTopics {
      * Get metadata of a partitioned topic asynchronously.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return a future that can be used to track when the partitioned topic metadata is returned
      */
-    CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination);
+    CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic);
 
     /**
      * Get the stats for the topic.
@@ -117,8 +117,8 @@ public interface NonPersistentTopics {
      *
      * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return the topic statistics
      *
      * @throws NotAuthorizedException
@@ -128,27 +128,27 @@ public interface NonPersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    NonPersistentTopicStats getStats(String destination) throws PulsarAdminException;
+    NonPersistentTopicStats getStats(String topic) throws PulsarAdminException;
 
     /**
      * Get the stats for the topic asynchronously. All the rates are computed over a 1 minute window and are relative
      * the last completed 1 minute period.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      *
      * @return a future that can be used to track when the topic statistics are returned
      *
      */
-    CompletableFuture<NonPersistentTopicStats> getStatsAsync(String destination);
+    CompletableFuture<NonPersistentTopicStats> getStatsAsync(String topic);
 
     /**
      * Get the internal stats for the topic.
      * <p>
      * Access the internal state of the topic
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return the topic statistics
      *
      * @throws NotAuthorizedException
@@ -158,17 +158,17 @@ public interface NonPersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException;
+    PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException;
 
     /**
      * Get the internal stats for the topic asynchronously.
      *
-     * @param destination
-     *            Destination Name
+     * @param topic
+     *            Topic Name
      *
      * @return a future that can be used to track when the internal topic statistics are returned
      */
-    CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination);
+    CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic);
 
     /**
      * Create a partitioned topic.
@@ -176,13 +176,13 @@ public interface NonPersistentTopics {
      * Create a partitioned topic. It needs to be called before creating a producer for a partitioned topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of partitions to create of the topic
      * @throws PulsarAdminException
      */
-    void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+    void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
 
     /**
      * Create a partitioned topic asynchronously.
@@ -191,44 +191,44 @@ public interface NonPersistentTopics {
      * topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of partitions to create of the topic
      * @return a future that can be used to track when the partitioned topic is created
      */
-    CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions);
-    
+    CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions);
+
     /**
      * Unload a topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      *
      * @throws NotAuthorizedException
      *             Don't have admin permission
      * @throws NotFoundException
-     *             Destination does not exist
+     *             Topic does not exist
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void unload(String destination) throws PulsarAdminException;
+    void unload(String topic) throws PulsarAdminException;
 
     /**
      * Unload a topic asynchronously.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      *
      * @return a future that can be used to track when the topic is unloaded
      */
-    CompletableFuture<Void> unloadAsync(String destination);
+    CompletableFuture<Void> unloadAsync(String topic);
 
     /**
      * Get list of topics exist into given bundle
-     * 
+     *
      * @param namespace
      * @param bundleRange
      * @return
@@ -239,7 +239,7 @@ public interface NonPersistentTopics {
 
     /**
      * Get list of topics exist into given bundle asynchronously.
-     * 
+     *
      * @param namespace
      * @param bundleRange
      * @return
@@ -248,7 +248,7 @@ public interface NonPersistentTopics {
 
     /**
      * Get list of topics exist into given namespace
-     * 
+     *
      * @param namespace
      * @return
      * @throws PulsarAdminException
@@ -257,7 +257,7 @@ public interface NonPersistentTopics {
 
     /**
      * Get list of topics exist into given namespace asynchronously.
-     * 
+     *
      * @param namespace
      * @param bundleRange
      * @return
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
index 70642fe..b36caf8 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
@@ -53,7 +53,7 @@ public interface PersistentTopics {
      *
      * @param namespace
      *            Namespace name
-     * @return a list of destinations
+     * @return a list of topics
      *
      * @throws NotAuthorizedException
      *             Don't have admin permission
@@ -88,10 +88,10 @@ public interface PersistentTopics {
     List<String> getPartitionedTopicList(String namespace) throws PulsarAdminException;
 
     /**
-     * Get permissions on a destination.
+     * Get permissions on a topic.
      * <p>
-     * Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the
-     * namespace level combined (union) with any eventual specific permission set on the destination.
+     * Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the
+     * namespace level combined (union) with any eventual specific permission set on the topic.
      * <p>
      * Response Example:
      *
@@ -102,9 +102,9 @@ public interface PersistentTopics {
      * }</code>
      * </pre>
      *
-     * @param destination
-     *            Destination url
-     * @return a map of destinations an their permissions set
+     * @param topic
+     *            Topic url
+     * @return a map of topics an their permissions set
      *
      * @throws NotAuthorizedException
      *             Don't have admin permission
@@ -113,12 +113,12 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    Map<String, Set<AuthAction>> getPermissions(String destination) throws PulsarAdminException;
+    Map<String, Set<AuthAction>> getPermissions(String topic) throws PulsarAdminException;
 
     /**
-     * Grant permission on a destination.
+     * Grant permission on a topic.
      * <p>
-     * Grant a new permission to a client role on a single destination.
+     * Grant a new permission to a client role on a single topic.
      * <p>
      * Request parameter example:
      *
@@ -126,8 +126,8 @@ public interface PersistentTopics {
      * <code>["produce", "consume"]</code>
      * </pre>
      *
-     * @param destination
-     *            Destination url
+     * @param topic
+     *            Topic url
      * @param role
      *            Client role to which grant permission
      * @param actions
@@ -142,16 +142,16 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void grantPermission(String destination, String role, Set<AuthAction> actions) throws PulsarAdminException;
+    void grantPermission(String topic, String role, Set<AuthAction> actions) throws PulsarAdminException;
 
     /**
-     * Revoke permissions on a destination.
+     * Revoke permissions on a topic.
      * <p>
-     * Revoke permissions to a client role on a single destination. If the permission was not set at the destination
-     * level, but rather at the namespace level, this operation will return an error (HTTP status code 412).
+     * Revoke permissions to a client role on a single topic. If the permission was not set at the topic level, but
+     * rather at the namespace level, this operation will return an error (HTTP status code 412).
      *
-     * @param destination
-     *            Destination url
+     * @param topic
+     *            Topic url
      * @param role
      *            Client role to which remove permission
      * @throws UniformInterfaceException
@@ -162,11 +162,11 @@ public interface PersistentTopics {
      * @throws NotFound
      *             Namespace does not exist
      * @throws PreconditionFailedException
-     *             Permissions are not set at the destination level
+     *             Permissions are not set at the topic level
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void revokePermissions(String destination, String role) throws PulsarAdminException;
+    void revokePermissions(String topic, String role) throws PulsarAdminException;
 
     /**
      * Create a partitioned topic.
@@ -174,13 +174,13 @@ public interface PersistentTopics {
      * Create a partitioned topic. It needs to be called before creating a producer for a partitioned topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of partitions to create of the topic
      * @throws PulsarAdminException
      */
-    void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+    void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
 
     /**
      * Create a partitioned topic asynchronously.
@@ -189,13 +189,13 @@ public interface PersistentTopics {
      * topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of partitions to create of the topic
      * @return a future that can be used to track when the partitioned topic is created
      */
-    CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions);
+    CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions);
 
     /**
      * Update number of partitions of a non-global partitioned topic.
@@ -204,14 +204,14 @@ public interface PersistentTopics {
      * number of partitions. Decrementing number of partitions requires deletion of topic which is not supported.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of new partitions of already exist partitioned-topic
      *
      * @return a future that can be used to track when the partitioned topic is updated
      */
-    void updatePartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+    void updatePartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
 
     /**
      * Update number of partitions of a non-global partitioned topic asynchronously.
@@ -220,14 +220,14 @@ public interface PersistentTopics {
      * number of partitions. Decrementing number of partitions requires deletion of topic which is not supported.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @param numPartitions
      *            Number of new partitions of already exist partitioned-topic
      *
      * @return a future that can be used to track when the partitioned topic is updated
      */
-    CompletableFuture<Void> updatePartitionedTopicAsync(String destination, int numPartitions);
+    CompletableFuture<Void> updatePartitionedTopicAsync(String topic, int numPartitions);
 
     /**
      * Get metadata of a partitioned topic.
@@ -235,12 +235,12 @@ public interface PersistentTopics {
      * Get metadata of a partitioned topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return Partitioned topic metadata
      * @throws PulsarAdminException
      */
-    PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException;
+    PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException;
 
     /**
      * Get metadata of a partitioned topic asynchronously.
@@ -248,11 +248,11 @@ public interface PersistentTopics {
      * Get metadata of a partitioned topic asynchronously.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return a future that can be used to track when the partitioned topic metadata is returned
      */
-    CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination);
+    CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic);
 
     /**
      * Delete a partitioned topic.
@@ -260,11 +260,11 @@ public interface PersistentTopics {
      * It will also delete all the partitions of the topic if it exists.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @throws PulsarAdminException
      */
-    void deletePartitionedTopic(String destination) throws PulsarAdminException;
+    void deletePartitionedTopic(String topic) throws PulsarAdminException;
 
     /**
      * Delete a partitioned topic asynchronously.
@@ -272,11 +272,11 @@ public interface PersistentTopics {
      * It will also delete all the partitions of the topic if it exists.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      * @return a future that can be used to track when the partitioned topic is deleted
      */
-    CompletableFuture<Void> deletePartitionedTopicAsync(String destination);
+    CompletableFuture<Void> deletePartitionedTopicAsync(String topic);
 
     /**
      * Delete a topic.
@@ -284,19 +284,19 @@ public interface PersistentTopics {
      * Delete a topic. The topic cannot be deleted if there's any active subscription or producer connected to the it.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            Topic name
      *
      * @throws NotAuthorizedException
      *             Don't have admin permission
      * @throws NotFoundException
-     *             Destination does not exist
+     *             Topic does not exist
      * @throws PreconditionFailedException
      *             Topic has active subscriptions or producers
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void delete(String destination) throws PulsarAdminException;
+    void delete(String topic) throws PulsarAdminException;
 
     /**
      * Delete a topic asynchronously.
@@ -305,50 +305,50 @@ public interface PersistentTopics {
      * connected to the it.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      *
      * @return a future that can be used to track when the topic is deleted
      */
-    CompletableFuture<Void> deleteAsync(String destination);
+    CompletableFuture<Void> deleteAsync(String topic);
 
     /**
      * Unload a topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      *
      * @throws NotAuthorizedException
      *             Don't have admin permission
      * @throws NotFoundException
-     *             Destination does not exist
+     *             topic does not exist
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void unload(String destination) throws PulsarAdminException;
+    void unload(String topic) throws PulsarAdminException;
 
     /**
      * Unload a topic asynchronously.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      *
      * @return a future that can be used to track when the topic is unloaded
      */
-    CompletableFuture<Void> unloadAsync(String destination);
+    CompletableFuture<Void> unloadAsync(String topic);
 
     /**
      * Terminate the topic and prevent any more messages being published on it.
      * <p>
      * This
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return the message id of the last message that was published in the topic
      */
-    CompletableFuture<MessageId> terminateTopicAsync(String destination);
+    CompletableFuture<MessageId> terminateTopicAsync(String topic);
 
     /**
      * Get the list of subscriptions.
@@ -356,8 +356,8 @@ public interface PersistentTopics {
      * Get the list of persistent subscriptions for a given topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return the list of subscriptions
      *
      * @throws NotAuthorizedException
@@ -367,7 +367,7 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    List<String> getSubscriptions(String destination) throws PulsarAdminException;
+    List<String> getSubscriptions(String topic) throws PulsarAdminException;
 
     /**
      * Get the list of subscriptions asynchronously.
@@ -375,11 +375,11 @@ public interface PersistentTopics {
      * Get the list of persistent subscriptions for a given topic.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return a future that can be used to track when the list of subscriptions is returned
      */
-    CompletableFuture<List<String>> getSubscriptionsAsync(String destination);
+    CompletableFuture<List<String>> getSubscriptionsAsync(String topic);
 
     /**
      * Get the stats for the topic.
@@ -444,8 +444,8 @@ public interface PersistentTopics {
      *
      * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return the topic statistics
      *
      * @throws NotAuthorizedException
@@ -455,27 +455,27 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    PersistentTopicStats getStats(String destination) throws PulsarAdminException;
+    PersistentTopicStats getStats(String topic) throws PulsarAdminException;
 
     /**
      * Get the stats for the topic asynchronously. All the rates are computed over a 1 minute window and are relative
      * the last completed 1 minute period.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      *
      * @return a future that can be used to track when the topic statistics are returned
      *
      */
-    CompletableFuture<PersistentTopicStats> getStatsAsync(String destination);
+    CompletableFuture<PersistentTopicStats> getStatsAsync(String topic);
 
     /**
      * Get the internal stats for the topic.
      * <p>
      * Access the internal state of the topic
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return the topic statistics
      *
      * @throws NotAuthorizedException
@@ -485,23 +485,23 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException;
+    PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException;
 
     /**
      * Get the internal stats for the topic asynchronously.
      *
-     * @param destination
-     *            Destination Name
+     * @param topic
+     *            topic Name
      *
      * @return a future that can be used to track when the internal topic statistics are returned
      */
-    CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination);
+    CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic);
 
     /**
      * Get a JSON representation of the topic metadata stored in ZooKeeper
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return the topic internal metadata
      * @throws NotAuthorizedException
      *             Don't have admin permission
@@ -510,13 +510,13 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    JsonObject getInternalInfo(String destination) throws PulsarAdminException;
+    JsonObject getInternalInfo(String topic) throws PulsarAdminException;
 
     /**
      * Get a JSON representation of the topic metadata stored in ZooKeeper
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @return a future to receive the topic internal metadata
      * @throws NotAuthorizedException
      *             Don't have admin permission
@@ -525,7 +525,7 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    CompletableFuture<JsonObject> getInternalInfoAsync(String destination);
+    CompletableFuture<JsonObject> getInternalInfoAsync(String topic);
 
     /**
      * Get the stats for the partitioned topic
@@ -584,8 +584,8 @@ public interface PersistentTopics {
      *
      * All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param perPartition
      *
      * @return the partitioned topic statistics
@@ -597,18 +597,18 @@ public interface PersistentTopics {
      *             Unexpected error
      *
      */
-    PartitionedTopicStats getPartitionedStats(String destination, boolean perPartition) throws PulsarAdminException;
+    PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition) throws PulsarAdminException;
 
     /**
      * Get the stats for the partitioned topic asynchronously
      *
-     * @param destination
-     *            Destination Name
+     * @param topic
+     *            topic Name
      * @param perPartition
      *            flag to get stats per partition
      * @return a future that can be used to track when the partitioned topic statistics are returned
      */
-    CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String destination, boolean perPartition);
+    CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String topic, boolean perPartition);
 
     /**
      * Delete a subscription.
@@ -616,8 +616,8 @@ public interface PersistentTopics {
      * Delete a persistent subscription from a topic. There should not be any active consumers on the subscription.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      *
@@ -630,7 +630,7 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void deleteSubscription(String destination, String subName) throws PulsarAdminException;
+    void deleteSubscription(String topic, String subName) throws PulsarAdminException;
 
     /**
      * Delete a subscription asynchronously.
@@ -638,22 +638,22 @@ public interface PersistentTopics {
      * Delete a persistent subscription from a topic. There should not be any active consumers on the subscription.
      * <p>
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      *
      * @return a future that can be used to track when the subscription is deleted
      */
-    CompletableFuture<Void> deleteSubscriptionAsync(String destination, String subName);
+    CompletableFuture<Void> deleteSubscriptionAsync(String topic, String subName);
 
     /**
      * Skip all messages on a topic subscription.
      * <p>
      * Completely clears the backlog on the subscription.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      *
@@ -664,27 +664,27 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void skipAllMessages(String destination, String subName) throws PulsarAdminException;
+    void skipAllMessages(String topic, String subName) throws PulsarAdminException;
 
     /**
      * Skip all messages on a topic subscription asynchronously.
      * <p>
      * Completely clears the backlog on the subscription.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      *
      * @return a future that can be used to track when all the messages are skipped
      */
-    CompletableFuture<Void> skipAllMessagesAsync(String destination, String subName);
+    CompletableFuture<Void> skipAllMessagesAsync(String topic, String subName);
 
     /**
      * Skip messages on a topic subscription.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param numMessages
@@ -697,13 +697,13 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void skipMessages(String destination, String subName, long numMessages) throws PulsarAdminException;
+    void skipMessages(String topic, String subName, long numMessages) throws PulsarAdminException;
 
     /**
      * Skip messages on a topic subscription asynchronously.
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param numMessages
@@ -711,13 +711,13 @@ public interface PersistentTopics {
      *
      * @return a future that can be used to track when the number of messages are skipped
      */
-    CompletableFuture<Void> skipMessagesAsync(String destination, String subName, long numMessages);
+    CompletableFuture<Void> skipMessagesAsync(String topic, String subName, long numMessages);
 
     /**
      * Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param expireTimeInSeconds
@@ -725,51 +725,53 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    public void expireMessages(String destination, String subscriptionName, long expireTimeInSeconds) throws PulsarAdminException;
+    public void expireMessages(String topic, String subscriptionName, long expireTimeInSeconds)
+            throws PulsarAdminException;
 
     /**
      * Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription asynchronously
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param expireTimeInSeconds
      *            Expire messages older than time in seconds
      * @return
      */
-    public CompletableFuture<Void> expireMessagesAsync(String destination, String subscriptionName, long expireTimeInSeconds);
+    public CompletableFuture<Void> expireMessagesAsync(String topic, String subscriptionName,
+            long expireTimeInSeconds);
 
     /**
      * Expire all messages older than given N (expireTimeInSeconds) seconds for all subscriptions of the
      * persistent-topic
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param expireTimeInSeconds
      *            Expire messages older than time in seconds
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    public void expireMessagesForAllSubscriptions(String destination, long expireTimeInSeconds) throws PulsarAdminException;
-
+    public void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds)
+            throws PulsarAdminException;
 
     /**
      * Expire all messages older than given N (expireTimeInSeconds) seconds for all subscriptions of the
      * persistent-topic asynchronously
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param expireTimeInSeconds
      *            Expire messages older than time in seconds
      */
-    public CompletableFuture<Void> expireMessagesForAllSubscriptionsAsync(String destination, long expireTimeInSeconds);
+    public CompletableFuture<Void> expireMessagesForAllSubscriptionsAsync(String topic, long expireTimeInSeconds);
 
     /**
      * Peek messages from a topic subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param numMessages
@@ -782,26 +784,26 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    List<Message> peekMessages(String destination, String subName, int numMessages) throws PulsarAdminException;
+    List<Message> peekMessages(String topic, String subName, int numMessages) throws PulsarAdminException;
 
     /**
      * Peek messages from a topic subscription asynchronously
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param numMessages
      *            Number of messages
      * @return a future that can be used to track when the messages are returned
      */
-    CompletableFuture<List<Message>> peekMessagesAsync(String destination, String subName, int numMessages);
+    CompletableFuture<List<Message>> peekMessagesAsync(String topic, String subName, int numMessages);
 
     /**
      * Create a new subscription on a topic
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subscriptionName
      *            Subscription name
      * @param messageId
@@ -817,27 +819,27 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void createSubscription(String destination, String subscriptionName, MessageId messageId)
+    void createSubscription(String topic, String subscriptionName, MessageId messageId)
             throws PulsarAdminException;
 
     /**
      * Create a new subscription on a topic
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subscriptionName
      *            Subscription name
      * @param messageId
      *            The {@link MessageId} on where to initialize the subscription. It could be {@link MessageId#latest},
      *            {@link MessageId#earliest} or a specific message id.
      */
-    CompletableFuture<Void> createSubscriptionAsync(String destination, String subscriptionName, MessageId messageId);
+    CompletableFuture<Void> createSubscriptionAsync(String topic, String subscriptionName, MessageId messageId);
 
     /**
      * Reset cursor position on a topic subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param timestamp
@@ -852,25 +854,25 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void resetCursor(String destination, String subName, long timestamp) throws PulsarAdminException;
+    void resetCursor(String topic, String subName, long timestamp) throws PulsarAdminException;
 
     /**
      * Reset cursor position on a topic subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param timestamp
      *            reset subscription to position closest to time in ms since epoch
      */
-    CompletableFuture<Void> resetCursorAsync(String destination, String subName, long timestamp);
+    CompletableFuture<Void> resetCursorAsync(String topic, String subName, long timestamp);
 
     /**
      * Reset cursor position on a topic subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param messageId
@@ -885,17 +887,17 @@ public interface PersistentTopics {
      * @throws PulsarAdminException
      *             Unexpected error
      */
-    void resetCursor(String destination, String subName, MessageId messageId) throws PulsarAdminException;
+    void resetCursor(String topic, String subName, MessageId messageId) throws PulsarAdminException;
 
     /**
      * Reset cursor position on a topic subscription
      *
-     * @param destination
-     *            Destination name
+     * @param topic
+     *            topic name
      * @param subName
      *            Subscription name
      * @param MessageId
      *            reset subscription to messageId (or previous nearest messageId if given messageId is not valid)
      */
-    CompletableFuture<Void> resetCursorAsync(String destination, String subName, MessageId messageId);
+    CompletableFuture<Void> resetCursorAsync(String topic, String subName, MessageId messageId);
 }
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
index e801925..9be2516 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
@@ -110,7 +110,7 @@ public interface Properties {
     /**
      * Delete an existing property.
      * <p>
-     * Delete a property and all namespaces and destinations under it.
+     * Delete a property and all namespaces and topics under it.
      *
      * @param property
      *            Property name
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
index 21f809d..dfa6579 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
@@ -264,14 +264,14 @@ public class PulsarAdmin implements Closeable {
     public ResourceQuotas resourceQuotas() {
         return resourceQuotas;
     }
-    
+
     /**
-     * @return does a looks up for the broker serving the destination
+     * @return does a looks up for the broker serving the topic
      */
     public Lookup lookups() {
         return lookups;
     }
-    
+
     /**
      * @return the broker statics
      */
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
index 917fd65..82e1722 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
@@ -75,7 +75,7 @@ public class BrokerStatsImpl extends BaseResource implements BrokerStats {
     }
 
     @Override
-    public JsonObject getDestinations() throws PulsarAdminException {
+    public JsonObject getTopics() throws PulsarAdminException {
         try {
             String json = request(brokerStats.path("/destinations")).get(String.class);
             return new Gson().fromJson(json, JsonObject.class);
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
index 4a943c9..677dd46 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
@@ -25,7 +25,7 @@ import org.apache.pulsar.client.admin.Lookup;
 import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 
 public class LookupImpl extends BaseResource implements Lookup {
 
@@ -47,26 +47,26 @@ public class LookupImpl extends BaseResource implements Lookup {
     }
 
     @Override
-    public String lookupDestination(String destination) throws PulsarAdminException {
+    public String lookupTopic(String topic) throws PulsarAdminException {
         try {
-            DestinationName destName = DestinationName.get(destination);
-            return doDestinationLookup(v2lookup.path("/destination"), destName);
+            TopicName topicName = TopicName.get(topic);
+            return doTopicLookup(v2lookup.path("/destination"), topicName);
         } catch (Exception e) {
             throw getLookupApiException(e);
         }
     }
 
     @Override
-    public String getBundleRange(String destination) throws PulsarAdminException {
+    public String getBundleRange(String topic) throws PulsarAdminException {
         try {
-            DestinationName destName = DestinationName.get(destination);
+            TopicName destName = TopicName.get(topic);
             return request(v2lookup.path("/destination").path(destName.getLookupName()).path("bundle")).get(String.class);
         } catch (Exception e) {
             throw getLookupApiException(e);
         }
     }
 
-    private String doDestinationLookup(WebTarget lookupResource, DestinationName destName) throws PulsarAdminException {
+    private String doTopicLookup(WebTarget lookupResource, TopicName destName) throws PulsarAdminException {
         LookupData lookupData = request(lookupResource.path(destName.getLookupName())).get(LookupData.class);
         if (useTls) {
             return lookupData.getBrokerUrlTls();
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 3f0c7e7..6db9ab0 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
@@ -72,7 +72,7 @@ public class NamespacesImpl extends BaseResource implements Namespaces {
     }
 
     @Override
-    public List<String> getDestinations(String namespace) throws PulsarAdminException {
+    public List<String> getTopics(String namespace) throws PulsarAdminException {
         try {
             NamespaceName ns = NamespaceName.get(namespace);
             return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
index bfaa81f..48bfd09 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
@@ -33,7 +33,7 @@ import javax.ws.rs.core.MediaType;
 import org.apache.pulsar.client.admin.NonPersistentTopics;
 import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.client.api.Authentication;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
@@ -49,9 +49,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+    public void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
         try {
-            createPartitionedTopicAsync(destination, numPartitions).get();
+            createPartitionedTopicAsync(topic, numPartitions).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -61,18 +61,18 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions) {
+    public CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions) {
         checkArgument(numPartitions > 1, "Number of partitions should be more than 1");
-        DestinationName ds = validateTopic(destination);
+        TopicName ds = validateTopic(topic);
         return asyncPutRequest(
                 nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
                 Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
     }
 
     @Override
-    public PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException {
+    public PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException {
         try {
-            return getPartitionedTopicMetadataAsync(destination).get();
+            return getPartitionedTopicMetadataAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -82,8 +82,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
         asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
                 new InvocationCallback<PartitionedTopicMetadata>() {
@@ -102,9 +102,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public NonPersistentTopicStats getStats(String destination) throws PulsarAdminException {
+    public NonPersistentTopicStats getStats(String topic) throws PulsarAdminException {
         try {
-            return getStatsAsync(destination).get();
+            return getStatsAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -114,8 +114,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public CompletableFuture<NonPersistentTopicStats> getStatsAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<NonPersistentTopicStats> getStatsAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<NonPersistentTopicStats> future = new CompletableFuture<>();
         asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("stats"),
                 new InvocationCallback<NonPersistentTopicStats>() {
@@ -134,9 +134,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException {
+    public PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException {
         try {
-            return getInternalStatsAsync(destination).get();
+            return getInternalStatsAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -146,8 +146,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PersistentTopicInternalStats> future = new CompletableFuture<>();
         asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internalStats"),
                 new InvocationCallback<PersistentTopicInternalStats>() {
@@ -166,9 +166,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public void unload(String destination) throws PulsarAdminException {
+    public void unload(String topic) throws PulsarAdminException {
         try {
-            unloadAsync(destination).get();
+            unloadAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -178,8 +178,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     @Override
-    public CompletableFuture<Void> unloadAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> unloadAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         return asyncPutRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("unload"),
                 Entity.entity("", MediaType.APPLICATION_JSON));
     }
@@ -246,11 +246,11 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
     }
 
     /*
-     * returns destination name with encoded Local Name
+     * returns topic name with encoded Local Name
      */
-    private DestinationName validateTopic(String destination) {
+    private TopicName validateTopic(String topic) {
         // Parsing will throw exception if name is not valid
-        return DestinationName.get(destination);
+        return TopicName.get(topic);
     }
 
 }
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 01ede00..92bd242 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
@@ -53,7 +53,7 @@ import org.apache.pulsar.common.api.Commands;
 import org.apache.pulsar.common.api.proto.PulsarApi;
 import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue;
 import org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import org.apache.pulsar.common.policies.data.AuthAction;
@@ -107,9 +107,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public Map<String, Set<AuthAction>> getPermissions(String destination) throws PulsarAdminException {
+    public Map<String, Set<AuthAction>> getPermissions(String topic) throws PulsarAdminException {
         try {
-            DestinationName ds = DestinationName.get(destination);
+            TopicName ds = TopicName.get(topic);
             return request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions")).get(
                     new GenericType<Map<String, Set<AuthAction>>>() {
                     });
@@ -119,9 +119,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void grantPermission(String destination, String role, Set<AuthAction> actions) throws PulsarAdminException {
+    public void grantPermission(String topic, String role, Set<AuthAction> actions) throws PulsarAdminException {
         try {
-            DestinationName ds = DestinationName.get(destination);
+            TopicName ds = TopicName.get(topic);
             request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions").path(role))
                     .post(Entity.entity(actions, MediaType.APPLICATION_JSON), ErrorData.class);
         } catch (Exception e) {
@@ -130,9 +130,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void revokePermissions(String destination, String role) throws PulsarAdminException {
+    public void revokePermissions(String topic, String role) throws PulsarAdminException {
         try {
-            DestinationName ds = DestinationName.get(destination);
+            TopicName ds = TopicName.get(topic);
             request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions").path(role))
                     .delete(ErrorData.class);
         } catch (Exception e) {
@@ -141,9 +141,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+    public void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
         try {
-            createPartitionedTopicAsync(destination, numPartitions).get();
+            createPartitionedTopicAsync(topic, numPartitions).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -153,18 +153,18 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions) {
+    public CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions) {
         checkArgument(numPartitions > 1, "Number of partitions should be more than 1");
-        DestinationName ds = validateTopic(destination);
+        TopicName ds = validateTopic(topic);
         return asyncPutRequest(
                 persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
                 Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
     }
 
     @Override
-    public void updatePartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+    public void updatePartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
         try {
-            updatePartitionedTopicAsync(destination, numPartitions).get();
+            updatePartitionedTopicAsync(topic, numPartitions).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -174,18 +174,18 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> updatePartitionedTopicAsync(String destination, int numPartitions) {
+    public CompletableFuture<Void> updatePartitionedTopicAsync(String topic, int numPartitions) {
         checkArgument(numPartitions > 1, "Number of partitions must be more than 1");
-        DestinationName ds = validateTopic(destination);
+        TopicName ds = validateTopic(topic);
         return asyncPostRequest(
                 persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
                 Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
     }
 
     @Override
-    public PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException {
+    public PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException {
         try {
-            return getPartitionedTopicMetadataAsync(destination).get();
+            return getPartitionedTopicMetadataAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -195,8 +195,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
         asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
                 new InvocationCallback<PartitionedTopicMetadata>() {
@@ -215,9 +215,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void deletePartitionedTopic(String destination) throws PulsarAdminException {
+    public void deletePartitionedTopic(String topic) throws PulsarAdminException {
         try {
-            deletePartitionedTopicAsync(destination).get();
+            deletePartitionedTopicAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -227,16 +227,16 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> deletePartitionedTopicAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> deletePartitionedTopicAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName())
                 .path("partitions"));
     }
 
     @Override
-    public void delete(String destination) throws PulsarAdminException {
+    public void delete(String topic) throws PulsarAdminException {
         try {
-            deleteAsync(destination).get();
+            deleteAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -246,15 +246,15 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> deleteAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> deleteAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()));
     }
 
     @Override
-    public void unload(String destination) throws PulsarAdminException {
+    public void unload(String topic) throws PulsarAdminException {
         try {
-            unloadAsync(destination).get();
+            unloadAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -264,16 +264,16 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> unloadAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> unloadAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         return asyncPutRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("unload"),
                 Entity.entity("", MediaType.APPLICATION_JSON));
     }
 
     @Override
-    public List<String> getSubscriptions(String destination) throws PulsarAdminException {
+    public List<String> getSubscriptions(String topic) throws PulsarAdminException {
         try {
-            return getSubscriptionsAsync(destination).get();
+            return getSubscriptionsAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -283,8 +283,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<List<String>> getSubscriptionsAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<List<String>> getSubscriptionsAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<List<String>> future = new CompletableFuture<>();
         asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscriptions"),
                 new InvocationCallback<List<String>>() {
@@ -303,9 +303,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public PersistentTopicStats getStats(String destination) throws PulsarAdminException {
+    public PersistentTopicStats getStats(String topic) throws PulsarAdminException {
         try {
-            return getStatsAsync(destination).get();
+            return getStatsAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -315,8 +315,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<PersistentTopicStats> getStatsAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<PersistentTopicStats> getStatsAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PersistentTopicStats> future = new CompletableFuture<>();
         asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("stats"),
                 new InvocationCallback<PersistentTopicStats>() {
@@ -335,9 +335,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException {
+    public PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException {
         try {
-            return getInternalStatsAsync(destination).get();
+            return getInternalStatsAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -347,8 +347,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PersistentTopicInternalStats> future = new CompletableFuture<>();
         asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internalStats"),
                 new InvocationCallback<PersistentTopicInternalStats>() {
@@ -367,9 +367,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public JsonObject getInternalInfo(String destination) throws PulsarAdminException {
+    public JsonObject getInternalInfo(String topic) throws PulsarAdminException {
         try {
-            return getInternalInfoAsync(destination).get();
+            return getInternalInfoAsync(topic).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -379,8 +379,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<JsonObject> getInternalInfoAsync(String destination) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<JsonObject> getInternalInfoAsync(String topic) {
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<JsonObject> future = new CompletableFuture<>();
         asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internal-info"),
                 new InvocationCallback<String>() {
@@ -399,10 +399,10 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public PartitionedTopicStats getPartitionedStats(String destination, boolean perPartition)
+    public PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition)
             throws PulsarAdminException {
         try {
-            return getPartitionedStatsAsync(destination, perPartition).get();
+            return getPartitionedStatsAsync(topic, perPartition).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -412,9 +412,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String destination,
+    public CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String topic,
             boolean perPartition) {
-        DestinationName ds = validateTopic(destination);
+        TopicName ds = validateTopic(topic);
         final CompletableFuture<PartitionedTopicStats> future = new CompletableFuture<>();
         asyncGetRequest(
                 persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitioned-stats"),
@@ -437,9 +437,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void deleteSubscription(String destination, String subName) throws PulsarAdminException {
+    public void deleteSubscription(String topic, String subName) throws PulsarAdminException {
         try {
-            deleteSubscriptionAsync(destination, subName).get();
+            deleteSubscriptionAsync(topic, subName).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -449,17 +449,17 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> deleteSubscriptionAsync(String destination, String subName) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> deleteSubscriptionAsync(String topic, String subName) {
+        TopicName ds = validateTopic(topic);
         String encodedSubName = Codec.encode(subName);
         return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName())
                 .path("subscription").path(encodedSubName));
     }
 
     @Override
-    public void skipAllMessages(String destination, String subName) throws PulsarAdminException {
+    public void skipAllMessages(String topic, String subName) throws PulsarAdminException {
         try {
-            skipAllMessagesAsync(destination, subName).get();
+            skipAllMessagesAsync(topic, subName).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -469,8 +469,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> skipAllMessagesAsync(String destination, String subName) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> skipAllMessagesAsync(String topic, String subName) {
+        TopicName ds = validateTopic(topic);
         String encodedSubName = Codec.encode(subName);
         return asyncPostRequest(
                 persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscription")
@@ -478,9 +478,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void skipMessages(String destination, String subName, long numMessages) throws PulsarAdminException {
+    public void skipMessages(String topic, String subName, long numMessages) throws PulsarAdminException {
         try {
-            skipMessagesAsync(destination, subName, numMessages).get();
+            skipMessagesAsync(topic, subName, numMessages).get();
         } catch (ExecutionException e) {
             throw (PulsarAdminException) e.getCause();
         } catch (InterruptedException e) {
@@ -490,8 +490,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public CompletableFuture<Void> skipMessagesAsync(String destination, String subName, long numMessages) {
-        DestinationName ds = validateTopic(destination);
+    public CompletableFuture<Void> skipMessagesAsync(String topic, String subName, long numMessages) {
+        TopicName ds = validateTopic(topic);
         String encodedSubName = Codec.encode(subName);
         return asyncPostRequest(
                 persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscription")
@@ -500,9 +500,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
     }
 
     @Override
-    public void expireMessages(String destination, String subName, long expireTimeInSeconds) throws PulsarAdminException {
+    public void expireMessages(String topic, String subName, long expireTimeInSeconds) throws PulsarAdminException {
         try {
-            expireMessagesAsync(destination, subName, expireTimeInSeconds).get();
+            expireMessagesAsync(topic, subName, expireTimeInSeconds).get();
         } catch (ExecutionException e) {
... 2847 lines suppressed ...

-- 
To stop receiving notification emails like this one, please contact
mmerli@apache.org.