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 6645341  Renamed destination to 'topic' in C++ code (#1281)
6645341 is described below

commit 6645341c4bda82655d2fbbc05938bd6d77820226
Author: Matteo Merli <mme...@apache.org>
AuthorDate: Sun Feb 25 20:57:54 2018 -0800

    Renamed destination to 'topic' in C++ code (#1281)
---
 pulsar-client-cpp/include/pulsar/Message.h         |   2 +-
 pulsar-client-cpp/include/pulsar/MessageBuilder.h  |   4 +-
 pulsar-client-cpp/lib/BinaryProtoLookupService.cc  |  63 ++++++-----
 pulsar-client-cpp/lib/BinaryProtoLookupService.h   |  14 +--
 pulsar-client-cpp/lib/ClientConnection.cc          |   9 +-
 pulsar-client-cpp/lib/ClientConnection.h           |   4 +-
 pulsar-client-cpp/lib/ClientImpl.cc                |  51 ++++-----
 pulsar-client-cpp/lib/ClientImpl.h                 |   8 +-
 pulsar-client-cpp/lib/ConsumerImpl.cc              |   2 +-
 pulsar-client-cpp/lib/HTTPLookupService.cc         |  21 ++--
 pulsar-client-cpp/lib/HTTPLookupService.h          |   2 +-
 pulsar-client-cpp/lib/LookupService.h              |  12 +--
 pulsar-client-cpp/lib/PartitionedConsumerImpl.cc   |  20 ++--
 pulsar-client-cpp/lib/PartitionedConsumerImpl.h    |   6 +-
 pulsar-client-cpp/lib/PartitionedProducerImpl.cc   |  11 +-
 pulsar-client-cpp/lib/PartitionedProducerImpl.h    |   8 +-
 pulsar-client-cpp/lib/ProducerImpl.cc              |   2 +-
 .../lib/{DestinationName.cc => TopicName.cc}       |  80 +++++++-------
 .../lib/{DestinationName.h => TopicName.h}         |  24 ++---
 pulsar-client-cpp/perf/PerfConsumer.cc             |   2 +-
 pulsar-client-cpp/python/pulsar.py                 |   8 +-
 pulsar-client-cpp/tests/BasicEndToEndTest.cc       |   2 +-
 pulsar-client-cpp/tests/BatchMessageTest.cc        |   2 +-
 pulsar-client-cpp/tests/BinaryLookupServiceTest.cc |   8 +-
 pulsar-client-cpp/tests/ConsumerStatsTest.cc       |   2 +-
 pulsar-client-cpp/tests/DestinationNameTest.cc     | 117 ---------------------
 pulsar-client-cpp/tests/TopicNameTest.cc           | 110 +++++++++++++++++++
 27 files changed, 292 insertions(+), 302 deletions(-)

diff --git a/pulsar-client-cpp/include/pulsar/Message.h 
b/pulsar-client-cpp/include/pulsar/Message.h
index 5462a68..b98b48b 100644
--- a/pulsar-client-cpp/include/pulsar/Message.h
+++ b/pulsar-client-cpp/include/pulsar/Message.h
@@ -107,7 +107,7 @@ class Message {
 
     /**
      * Get the partition key for this message
-     * @return key string that is hashed to determine message's destination 
partition
+     * @return key string that is hashed to determine message's topic partition
      */
     const std::string& getPartitionKey() const;
     bool hasPartitionKey() const;
diff --git a/pulsar-client-cpp/include/pulsar/MessageBuilder.h 
b/pulsar-client-cpp/include/pulsar/MessageBuilder.h
index 7a08817..9738afe 100644
--- a/pulsar-client-cpp/include/pulsar/MessageBuilder.h
+++ b/pulsar-client-cpp/include/pulsar/MessageBuilder.h
@@ -65,7 +65,7 @@ class MessageBuilder {
 
     /*
      * set partition key for the message routing
-     * @param hash of this key is used to determine message's destination 
partition
+     * @param hash of this key is used to determine message's topic partition
      */
     MessageBuilder& setPartitionKey(const std::string& partitionKey);
 
@@ -95,7 +95,7 @@ class MessageBuilder {
     /**
      * override namespace replication clusters.  note that it is the
      * caller's responsibility to provide valid cluster names, and that
-     * all clusters have been previously configured as destinations.
+     * all clusters have been previously configured as topics.
      *
      * given an empty list, the message will replicate per the namespace
      * configuration.
diff --git a/pulsar-client-cpp/lib/BinaryProtoLookupService.cc 
b/pulsar-client-cpp/lib/BinaryProtoLookupService.cc
index e5b8d42..c4bef30 100644
--- a/pulsar-client-cpp/lib/BinaryProtoLookupService.cc
+++ b/pulsar-client-cpp/lib/BinaryProtoLookupService.cc
@@ -16,11 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-#include "DestinationName.h"
 #include "BinaryProtoLookupService.h"
 #include "SharedBuffer.h"
 
 #include <boost/shared_ptr.hpp>
+#include <lib/TopicName.h>
+
 #include <boost/bind.hpp>
 #include "ConnectionPool.h"
 
@@ -38,20 +39,19 @@ 
BinaryProtoLookupService::BinaryProtoLookupService(ConnectionPool& cnxPool, cons
     : cnxPool_(cnxPool), serviceUrl_(lookupUrl), mutex_(), 
requestIdGenerator_(0) {}
 
 /*
- * @param destination_name topic name to get broker for
+ * @param topicName topic name to get broker for
  *
- * Looks up the owner broker for the given destination name
+ * Looks up the owner broker for the given topic name
  */
-Future<Result, LookupDataResultPtr> BinaryProtoLookupService::lookupAsync(
-    const std::string& destinationName) {
-    DestinationNamePtr dn = DestinationName::get(destinationName);
-    if (!dn) {
-        LOG_ERROR("Unable to parse destination - " << destinationName);
+Future<Result, LookupDataResultPtr> 
BinaryProtoLookupService::lookupAsync(const std::string& topic) {
+    TopicNamePtr topicName = TopicName::get(topic);
+    if (!topicName) {
+        LOG_ERROR("Unable to parse topic - " << topic);
         LookupDataResultPromisePtr promise = 
boost::make_shared<LookupDataResultPromise>();
         promise->setFailed(ResultInvalidTopicName);
         return promise->getFuture();
     }
-    std::string lookupName = dn->toString();
+    std::string lookupName = topicName->toString();
     LookupDataResultPromisePtr promise = 
boost::make_shared<LookupDataResultPromise>();
     Future<Result, ClientConnectionWeakPtr> future = 
cnxPool_.getConnectionAsync(serviceUrl_, serviceUrl_);
     
future.addListener(boost::bind(&BinaryProtoLookupService::sendTopicLookupRequest,
 this, lookupName, false,
@@ -60,24 +60,24 @@ Future<Result, LookupDataResultPtr> 
BinaryProtoLookupService::lookupAsync(
 }
 
 /*
- * @param    destination_name topic to get number of partitions.
+ * @param    topicName topic to get number of partitions.
  *
  */
 Future<Result, LookupDataResultPtr> 
BinaryProtoLookupService::getPartitionMetadataAsync(
-    const DestinationNamePtr& dn) {
+    const TopicNamePtr& topicName) {
     LookupDataResultPromisePtr promise = 
boost::make_shared<LookupDataResultPromise>();
-    if (!dn) {
+    if (!topicName) {
         promise->setFailed(ResultInvalidTopicName);
         return promise->getFuture();
     }
-    std::string lookupName = dn->toString();
+    std::string lookupName = topicName->toString();
     Future<Result, ClientConnectionWeakPtr> future = 
cnxPool_.getConnectionAsync(serviceUrl_, serviceUrl_);
     
future.addListener(boost::bind(&BinaryProtoLookupService::sendPartitionMetadataLookupRequest,
 this,
                                    lookupName, _1, _2, promise));
     return promise->getFuture();
 }
 
-void BinaryProtoLookupService::sendTopicLookupRequest(const std::string& 
destinationName, bool authoritative,
+void BinaryProtoLookupService::sendTopicLookupRequest(const std::string& 
topicName, bool authoritative,
                                                       Result result, const 
ClientConnectionWeakPtr& clientCnx,
                                                       
LookupDataResultPromisePtr promise) {
     if (result != ResultOk) {
@@ -87,40 +87,37 @@ void BinaryProtoLookupService::sendTopicLookupRequest(const 
std::string& destina
     LookupDataResultPromisePtr lookupPromise = 
boost::make_shared<LookupDataResultPromise>();
     ClientConnectionPtr conn = clientCnx.lock();
     uint64_t requestId = newRequestId();
-    conn->newTopicLookup(destinationName, authoritative, requestId, 
lookupPromise);
-    
lookupPromise->getFuture().addListener(boost::bind(&BinaryProtoLookupService::handleLookup,
 this,
-                                                       destinationName, _1, 
_2, clientCnx, promise));
+    conn->newTopicLookup(topicName, authoritative, requestId, lookupPromise);
+    lookupPromise->getFuture().addListener(
+        boost::bind(&BinaryProtoLookupService::handleLookup, this, topicName, 
_1, _2, clientCnx, promise));
 }
 
-void BinaryProtoLookupService::handleLookup(const std::string& 
destinationName, Result result,
+void BinaryProtoLookupService::handleLookup(const std::string& topicName, 
Result result,
                                             LookupDataResultPtr data,
                                             const ClientConnectionWeakPtr& 
clientCnx,
                                             LookupDataResultPromisePtr 
promise) {
     if (data) {
         if (data->isRedirect()) {
-            LOG_DEBUG("Lookup request is for " << destinationName << " 
redirected to "
-                                               << data->getBrokerUrl());
+            LOG_DEBUG("Lookup request is for " << topicName << " redirected to 
" << data->getBrokerUrl());
 
             const std::string& logicalAddress = data->getBrokerUrl();
             const std::string& physicalAddress =
                 data->shouldProxyThroughServiceUrl() ? serviceUrl_ : 
logicalAddress;
             Future<Result, ClientConnectionWeakPtr> future =
                 cnxPool_.getConnectionAsync(logicalAddress, physicalAddress);
-            
future.addListener(boost::bind(&BinaryProtoLookupService::sendTopicLookupRequest,
 this,
-                                           destinationName, 
data->isAuthoritative(), _1, _2, promise));
+            
future.addListener(boost::bind(&BinaryProtoLookupService::sendTopicLookupRequest,
 this, topicName,
+                                           data->isAuthoritative(), _1, _2, 
promise));
         } else {
-            LOG_DEBUG("Lookup response for " << destinationName << ", 
lookup-broker-url "
-                                             << data->getBrokerUrl());
+            LOG_DEBUG("Lookup response for " << topicName << ", 
lookup-broker-url " << data->getBrokerUrl());
             promise->setValue(data);
         }
     } else {
-        LOG_DEBUG("Lookup failed for " << destinationName << ", result " << 
result);
+        LOG_DEBUG("Lookup failed for " << topicName << ", result " << result);
         promise->setFailed(result);
     }
 }
 
-void BinaryProtoLookupService::sendPartitionMetadataLookupRequest(const 
std::string& destinationName,
-                                                                  Result 
result,
+void BinaryProtoLookupService::sendPartitionMetadataLookupRequest(const 
std::string& topicName, Result result,
                                                                   const 
ClientConnectionWeakPtr& clientCnx,
                                                                   
LookupDataResultPromisePtr promise) {
     if (result != ResultOk) {
@@ -131,22 +128,22 @@ void 
BinaryProtoLookupService::sendPartitionMetadataLookupRequest(const std::str
     LookupDataResultPromisePtr lookupPromise = 
boost::make_shared<LookupDataResultPromise>();
     ClientConnectionPtr conn = clientCnx.lock();
     uint64_t requestId = newRequestId();
-    conn->newPartitionedMetadataLookup(destinationName, requestId, 
lookupPromise);
+    conn->newPartitionedMetadataLookup(topicName, requestId, lookupPromise);
     lookupPromise->getFuture().addListener(
-        boost::bind(&BinaryProtoLookupService::handlePartitionMetadataLookup, 
this, destinationName, _1, _2,
+        boost::bind(&BinaryProtoLookupService::handlePartitionMetadataLookup, 
this, topicName, _1, _2,
                     clientCnx, promise));
 }
 
-void BinaryProtoLookupService::handlePartitionMetadataLookup(const 
std::string& destinationName,
-                                                             Result result, 
LookupDataResultPtr data,
+void BinaryProtoLookupService::handlePartitionMetadataLookup(const 
std::string& topicName, Result result,
+                                                             
LookupDataResultPtr data,
                                                              const 
ClientConnectionWeakPtr& clientCnx,
                                                              
LookupDataResultPromisePtr promise) {
     if (data) {
-        LOG_DEBUG("PartitionMetadataLookup response for " << destinationName 
<< ", lookup-broker-url "
+        LOG_DEBUG("PartitionMetadataLookup response for " << topicName << ", 
lookup-broker-url "
                                                           << 
data->getBrokerUrl());
         promise->setValue(data);
     } else {
-        LOG_DEBUG("PartitionMetadataLookup failed for " << destinationName << 
", result " << result);
+        LOG_DEBUG("PartitionMetadataLookup failed for " << topicName << ", 
result " << result);
         promise->setFailed(result);
     }
 }
diff --git a/pulsar-client-cpp/lib/BinaryProtoLookupService.h 
b/pulsar-client-cpp/lib/BinaryProtoLookupService.h
index 26dd9a0..f647c62 100644
--- a/pulsar-client-cpp/lib/BinaryProtoLookupService.h
+++ b/pulsar-client-cpp/lib/BinaryProtoLookupService.h
@@ -36,9 +36,9 @@ class BinaryProtoLookupService : public LookupService {
      */
     BinaryProtoLookupService(ConnectionPool& cnxPool, const std::string& 
serviceUrl);
 
-    Future<Result, LookupDataResultPtr> lookupAsync(const std::string& 
destinationName);
+    Future<Result, LookupDataResultPtr> lookupAsync(const std::string& 
topicName);
 
-    Future<Result, LookupDataResultPtr> getPartitionMetadataAsync(const 
DestinationNamePtr& dn);
+    Future<Result, LookupDataResultPtr> getPartitionMetadataAsync(const 
TopicNamePtr& topicName);
 
    private:
     boost::mutex mutex_;
@@ -47,18 +47,18 @@ class BinaryProtoLookupService : public LookupService {
     std::string serviceUrl_;
     ConnectionPool& cnxPool_;
 
-    void sendTopicLookupRequest(const std::string& destinationName, bool 
authoritative, Result result,
+    void sendTopicLookupRequest(const std::string& topicName, bool 
authoritative, Result result,
                                 const ClientConnectionWeakPtr& clientCnx, 
LookupDataResultPromisePtr promise);
 
-    void handleLookup(const std::string& destinationName, Result result, 
LookupDataResultPtr data,
+    void handleLookup(const std::string& topicName, Result result, 
LookupDataResultPtr data,
                       const ClientConnectionWeakPtr& clientCnx, 
LookupDataResultPromisePtr promise);
 
-    void sendPartitionMetadataLookupRequest(const std::string& 
destinationName, Result result,
+    void sendPartitionMetadataLookupRequest(const std::string& topicName, 
Result result,
                                             const ClientConnectionWeakPtr& 
clientCnx,
                                             LookupDataResultPromisePtr 
promise);
 
-    void handlePartitionMetadataLookup(const std::string& destinationName, 
Result result,
-                                       LookupDataResultPtr data, const 
ClientConnectionWeakPtr& clientCnx,
+    void handlePartitionMetadataLookup(const std::string& topicName, Result 
result, LookupDataResultPtr data,
+                                       const ClientConnectionWeakPtr& 
clientCnx,
                                        LookupDataResultPromisePtr promise);
 
     uint64_t newRequestId();
diff --git a/pulsar-client-cpp/lib/ClientConnection.cc 
b/pulsar-client-cpp/lib/ClientConnection.cc
index 018f4ab..fc0f611 100644
--- a/pulsar-client-cpp/lib/ClientConnection.cc
+++ b/pulsar-client-cpp/lib/ClientConnection.cc
@@ -953,15 +953,14 @@ Future<Result, BrokerConsumerStatsImpl> 
ClientConnection::newConsumerStats(uint6
     return promise.getFuture();
 }
 
-void ClientConnection::newTopicLookup(const std::string& destinationName, bool 
authoritative,
+void ClientConnection::newTopicLookup(const std::string& topicName, bool 
authoritative,
                                       const uint64_t requestId, 
LookupDataResultPromisePtr promise) {
-    newLookup(Commands::newLookup(destinationName, authoritative, requestId), 
requestId, promise);
+    newLookup(Commands::newLookup(topicName, authoritative, requestId), 
requestId, promise);
 }
 
-void ClientConnection::newPartitionedMetadataLookup(const std::string& 
destinationName,
-                                                    const uint64_t requestId,
+void ClientConnection::newPartitionedMetadataLookup(const std::string& 
topicName, const uint64_t requestId,
                                                     LookupDataResultPromisePtr 
promise) {
-    newLookup(Commands::newPartitionMetadataRequest(destinationName, 
requestId), requestId, promise);
+    newLookup(Commands::newPartitionMetadataRequest(topicName, requestId), 
requestId, promise);
 }
 
 void ClientConnection::newLookup(const SharedBuffer& cmd, const uint64_t 
requestId,
diff --git a/pulsar-client-cpp/lib/ClientConnection.h 
b/pulsar-client-cpp/lib/ClientConnection.h
index 47f4994..2176e27 100644
--- a/pulsar-client-cpp/lib/ClientConnection.h
+++ b/pulsar-client-cpp/lib/ClientConnection.h
@@ -111,10 +111,10 @@ class ClientConnection : public 
boost::enable_shared_from_this<ClientConnection>
 
     Future<Result, ClientConnectionWeakPtr> getCloseFuture();
 
-    void newTopicLookup(const std::string& destinationName, bool 
authoritative, const uint64_t requestId,
+    void newTopicLookup(const std::string& topicName, bool authoritative, 
const uint64_t requestId,
                         LookupDataResultPromisePtr promise);
 
-    void newPartitionedMetadataLookup(const std::string& destinationName, 
const uint64_t requestId,
+    void newPartitionedMetadataLookup(const std::string& topicName, const 
uint64_t requestId,
                                       LookupDataResultPromisePtr promise);
 
     void sendCommand(const SharedBuffer& cmd);
diff --git a/pulsar-client-cpp/lib/ClientImpl.cc 
b/pulsar-client-cpp/lib/ClientImpl.cc
index b760135..68d366a 100644
--- a/pulsar-client-cpp/lib/ClientImpl.cc
+++ b/pulsar-client-cpp/lib/ClientImpl.cc
@@ -22,7 +22,6 @@
 #include "ConsumerImpl.h"
 #include "ProducerImpl.h"
 #include "ReaderImpl.h"
-#include "DestinationName.h"
 #include "PartitionedProducerImpl.h"
 #include "PartitionedConsumerImpl.h"
 #include <boost/bind.hpp>
@@ -31,6 +30,7 @@
 #include <openssl/sha.h>
 #include "boost/date_time/posix_time/posix_time.hpp"
 #include <lib/HTTPLookupService.h>
+#include <lib/TopicName.h>
 
 DECLARE_LOG_OBJECT()
 
@@ -99,33 +99,33 @@ ExecutorServiceProviderPtr 
ClientImpl::getPartitionListenerExecutorProvider() {
 }
 void ClientImpl::createProducerAsync(const std::string& topic, 
ProducerConfiguration conf,
                                      CreateProducerCallback callback) {
-    DestinationNamePtr dn;
+    TopicNamePtr topicName;
     {
         Lock lock(mutex_);
         if (state_ != Open) {
             lock.unlock();
             callback(ResultAlreadyClosed, Producer());
             return;
-        } else if (!(dn = DestinationName::get(topic))) {
+        } else if (!(topicName = TopicName::get(topic))) {
             lock.unlock();
             callback(ResultInvalidTopicName, Producer());
             return;
         }
     }
-    lookupServicePtr_->getPartitionMetadataAsync(dn).addListener(
-        boost::bind(&ClientImpl::handleCreateProducer, shared_from_this(), _1, 
_2, dn, conf, callback));
+    
lookupServicePtr_->getPartitionMetadataAsync(topicName).addListener(boost::bind(
+        &ClientImpl::handleCreateProducer, shared_from_this(), _1, _2, 
topicName, conf, callback));
 }
 
 void ClientImpl::handleCreateProducer(const Result result, const 
LookupDataResultPtr partitionMetadata,
-                                      DestinationNamePtr dn, 
ProducerConfiguration conf,
+                                      TopicNamePtr topicName, 
ProducerConfiguration conf,
                                       CreateProducerCallback callback) {
     if (!result) {
         ProducerImplBasePtr producer;
         if (partitionMetadata->getPartitions() > 1) {
-            producer = 
boost::make_shared<PartitionedProducerImpl>(shared_from_this(), dn,
+            producer = 
boost::make_shared<PartitionedProducerImpl>(shared_from_this(), topicName,
                                                                    
partitionMetadata->getPartitions(), conf);
         } else {
-            producer = boost::make_shared<ProducerImpl>(shared_from_this(), 
dn->toString(), conf);
+            producer = boost::make_shared<ProducerImpl>(shared_from_this(), 
topicName->toString(), conf);
         }
         producer->getProducerCreatedFuture().addListener(
             boost::bind(&ClientImpl::handleProducerCreated, 
shared_from_this(), _1, _2, callback, producer));
@@ -135,7 +135,7 @@ void ClientImpl::handleCreateProducer(const Result result, 
const LookupDataResul
         producer->start();
     } else {
         LOG_ERROR("Error Checking/Getting Partition Metadata while creating 
producer on "
-                  << dn->toString() << " -- " << result);
+                  << topicName->toString() << " -- " << result);
         callback(result, Producer());
     }
 }
@@ -147,14 +147,14 @@ void ClientImpl::handleProducerCreated(Result result, 
ProducerImplBaseWeakPtr pr
 
 void ClientImpl::createReaderAsync(const std::string& topic, const MessageId& 
startMessageId,
                                    const ReaderConfiguration& conf, 
ReaderCallback callback) {
-    DestinationNamePtr dn;
+    TopicNamePtr topicName;
     {
         Lock lock(mutex_);
         if (state_ != Open) {
             lock.unlock();
             callback(ResultAlreadyClosed, Reader());
             return;
-        } else if (!(dn = DestinationName::get(topic))) {
+        } else if (!(topicName = TopicName::get(topic))) {
             lock.unlock();
             callback(ResultInvalidTopicName, Reader());
             return;
@@ -162,12 +162,13 @@ void ClientImpl::createReaderAsync(const std::string& 
topic, const MessageId& st
     }
 
     BatchMessageId msgId(startMessageId);
-    lookupServicePtr_->getPartitionMetadataAsync(dn).addListener(boost::bind(
-        &ClientImpl::handleReaderMetadataLookup, shared_from_this(), _1, _2, 
dn, msgId, conf, callback));
+    lookupServicePtr_->getPartitionMetadataAsync(topicName).addListener(
+        boost::bind(&ClientImpl::handleReaderMetadataLookup, 
shared_from_this(), _1, _2, topicName, msgId,
+                    conf, callback));
 }
 
 void ClientImpl::handleReaderMetadataLookup(const Result result, const 
LookupDataResultPtr partitionMetadata,
-                                            DestinationNamePtr dn, 
BatchMessageId startMessageId,
+                                            TopicNamePtr topicName, 
BatchMessageId startMessageId,
                                             ReaderConfiguration conf, 
ReaderCallback callback) {
     if (result != ResultOk) {
         LOG_ERROR("Error Checking/Getting Partition Metadata while creating 
reader: " << result);
@@ -176,12 +177,12 @@ void ClientImpl::handleReaderMetadataLookup(const Result 
result, const LookupDat
     }
 
     if (partitionMetadata->getPartitions() > 1) {
-        LOG_ERROR("Topic reader cannot be created on a partitioned topic: " << 
dn->toString());
+        LOG_ERROR("Topic reader cannot be created on a partitioned topic: " << 
topicName->toString());
         callback(ResultOperationNotSupported, Reader());
         return;
     }
 
-    ReaderImplPtr reader = boost::make_shared<ReaderImpl>(shared_from_this(), 
dn->toString(), conf,
+    ReaderImplPtr reader = boost::make_shared<ReaderImpl>(shared_from_this(), 
topicName->toString(), conf,
                                                           
getListenerExecutorProvider()->get(), callback);
     reader->start(startMessageId);
 
@@ -191,26 +192,26 @@ void ClientImpl::handleReaderMetadataLookup(const Result 
result, const LookupDat
 
 void ClientImpl::subscribeAsync(const std::string& topic, const std::string& 
consumerName,
                                 const ConsumerConfiguration& conf, 
SubscribeCallback callback) {
-    DestinationNamePtr dn;
+    TopicNamePtr topicName;
     {
         Lock lock(mutex_);
         if (state_ != Open) {
             lock.unlock();
             callback(ResultAlreadyClosed, Consumer());
             return;
-        } else if (!(dn = DestinationName::get(topic))) {
+        } else if (!(topicName = TopicName::get(topic))) {
             lock.unlock();
             callback(ResultInvalidTopicName, Consumer());
             return;
         }
     }
 
-    lookupServicePtr_->getPartitionMetadataAsync(dn).addListener(boost::bind(
-        &ClientImpl::handleSubscribe, shared_from_this(), _1, _2, dn, 
consumerName, conf, callback));
+    
lookupServicePtr_->getPartitionMetadataAsync(topicName).addListener(boost::bind(
+        &ClientImpl::handleSubscribe, shared_from_this(), _1, _2, topicName, 
consumerName, conf, callback));
 }
 
 void ClientImpl::handleSubscribe(const Result result, const 
LookupDataResultPtr partitionMetadata,
-                                 DestinationNamePtr dn, const std::string& 
consumerName,
+                                 TopicNamePtr topicName, const std::string& 
consumerName,
                                  ConsumerConfiguration conf, SubscribeCallback 
callback) {
     if (result == ResultOk) {
         // generate random name if not supplied by the customer.
@@ -224,11 +225,11 @@ void ClientImpl::handleSubscribe(const Result result, 
const LookupDataResultPtr
                 callback(ResultInvalidConfiguration, Consumer());
                 return;
             }
-            consumer = 
boost::make_shared<PartitionedConsumerImpl>(shared_from_this(), consumerName, 
dn,
-                                                                   
partitionMetadata->getPartitions(), conf);
+            consumer = boost::make_shared<PartitionedConsumerImpl>(
+                shared_from_this(), consumerName, topicName, 
partitionMetadata->getPartitions(), conf);
         } else {
-            consumer =
-                boost::make_shared<ConsumerImpl>(shared_from_this(), 
dn->toString(), consumerName, conf);
+            consumer = boost::make_shared<ConsumerImpl>(shared_from_this(), 
topicName->toString(),
+                                                        consumerName, conf);
         }
         consumer->getConsumerCreatedFuture().addListener(
             boost::bind(&ClientImpl::handleConsumerCreated, 
shared_from_this(), _1, _2, callback, consumer));
diff --git a/pulsar-client-cpp/lib/ClientImpl.h 
b/pulsar-client-cpp/lib/ClientImpl.h
index 679e2de..105df8a 100644
--- a/pulsar-client-cpp/lib/ClientImpl.h
+++ b/pulsar-client-cpp/lib/ClientImpl.h
@@ -24,9 +24,9 @@
 #include "BinaryProtoLookupService.h"
 #include "ConnectionPool.h"
 #include "LookupDataResult.h"
-#include "DestinationName.h"
 #include <boost/shared_ptr.hpp>
 #include <boost/thread/mutex.hpp>
+#include <lib/TopicName.h>
 #include "ProducerImplBase.h"
 #include "ConsumerImplBase.h"
 
@@ -83,15 +83,15 @@ class ClientImpl : public 
boost::enable_shared_from_this<ClientImpl> {
 
    private:
     void handleCreateProducer(const Result result, const LookupDataResultPtr 
partitionMetadata,
-                              DestinationNamePtr dn, ProducerConfiguration 
conf,
+                              TopicNamePtr topicName, ProducerConfiguration 
conf,
                               CreateProducerCallback callback);
 
     void handleSubscribe(const Result result, const LookupDataResultPtr 
partitionMetadata,
-                         DestinationNamePtr dn, const std::string& 
consumerName, ConsumerConfiguration conf,
+                         TopicNamePtr topicName, const std::string& 
consumerName, ConsumerConfiguration conf,
                          SubscribeCallback callback);
 
     void handleReaderMetadataLookup(const Result result, const 
LookupDataResultPtr partitionMetadata,
-                                    DestinationNamePtr dn, BatchMessageId 
startMessageId,
+                                    TopicNamePtr topicName, BatchMessageId 
startMessageId,
                                     ReaderConfiguration conf, ReaderCallback 
callback);
 
     void handleProducerCreated(Result result, ProducerImplBaseWeakPtr 
producerWeakPtr,
diff --git a/pulsar-client-cpp/lib/ConsumerImpl.cc 
b/pulsar-client-cpp/lib/ConsumerImpl.cc
index 1c62023..55e4e6c 100644
--- a/pulsar-client-cpp/lib/ConsumerImpl.cc
+++ b/pulsar-client-cpp/lib/ConsumerImpl.cc
@@ -21,11 +21,11 @@
 #include "Commands.h"
 #include "LogUtils.h"
 #include <boost/bind.hpp>
+#include <lib/TopicName.h>
 #include "pulsar/Result.h"
 #include "pulsar/MessageId.h"
 #include "Utils.h"
 #include <exception>
-#include "DestinationName.h"
 #include <algorithm>
 
 namespace pulsar {
diff --git a/pulsar-client-cpp/lib/HTTPLookupService.cc 
b/pulsar-client-cpp/lib/HTTPLookupService.cc
index 1a98f60..30b2af6 100644
--- a/pulsar-client-cpp/lib/HTTPLookupService.cc
+++ b/pulsar-client-cpp/lib/HTTPLookupService.cc
@@ -43,30 +43,31 @@ HTTPLookupService::HTTPLookupService(const std::string 
&lookupUrl,
     }
 }
 
-Future<Result, LookupDataResultPtr> HTTPLookupService::lookupAsync(const 
std::string &destinationName) {
+Future<Result, LookupDataResultPtr> HTTPLookupService::lookupAsync(const 
std::string &topic) {
     LookupPromise promise;
-    boost::shared_ptr<DestinationName> dn = 
DestinationName::get(destinationName);
-    if (!dn) {
-        LOG_ERROR("Unable to parse destination - " << destinationName);
+    boost::shared_ptr<TopicName> topicName = TopicName::get(topic);
+    if (!topicName) {
+        LOG_ERROR("Unable to parse topic - " << topic);
         promise.setFailed(ResultInvalidTopicName);
         return promise.getFuture();
     }
 
     std::stringstream completeUrlStream;
-    completeUrlStream << adminUrl_ << V2_PATH << "persistent/" << 
dn->getProperty() << '/' << dn->getCluster()
-                      << '/' << dn->getNamespacePortion() << '/' << 
dn->getEncodedLocalName();
+    completeUrlStream << adminUrl_ << V2_PATH << "persistent/" << 
topicName->getProperty() << '/'
+                      << topicName->getCluster() << '/' << 
topicName->getNamespacePortion() << '/'
+                      << topicName->getEncodedLocalName();
     
executorProvider_->get()->postWork(boost::bind(&HTTPLookupService::sendHTTPRequest,
 shared_from_this(),
                                                    promise, 
completeUrlStream.str(), Lookup));
     return promise.getFuture();
 }
 
 Future<Result, LookupDataResultPtr> 
HTTPLookupService::getPartitionMetadataAsync(
-    const DestinationNamePtr &dn) {
+    const TopicNamePtr &topicName) {
     LookupPromise promise;
     std::stringstream completeUrlStream;
-    completeUrlStream << adminUrl_ << PARTITION_PATH << dn->getProperty() << 
'/' << dn->getCluster() << '/'
-                      << dn->getNamespacePortion() << '/' << 
dn->getEncodedLocalName() << '/'
-                      << PARTITION_METHOD_NAME;
+    completeUrlStream << adminUrl_ << PARTITION_PATH << 
topicName->getProperty() << '/'
+                      << topicName->getCluster() << '/' << 
topicName->getNamespacePortion() << '/'
+                      << topicName->getEncodedLocalName() << '/' << 
PARTITION_METHOD_NAME;
     
executorProvider_->get()->postWork(boost::bind(&HTTPLookupService::sendHTTPRequest,
 shared_from_this(),
                                                    promise, 
completeUrlStream.str(), PartitionMetaData));
     return promise.getFuture();
diff --git a/pulsar-client-cpp/lib/HTTPLookupService.h 
b/pulsar-client-cpp/lib/HTTPLookupService.h
index b8c06e8..d7fff33 100644
--- a/pulsar-client-cpp/lib/HTTPLookupService.h
+++ b/pulsar-client-cpp/lib/HTTPLookupService.h
@@ -59,7 +59,7 @@ class HTTPLookupService : public LookupService, public 
boost::enable_shared_from
 
     Future<Result, LookupDataResultPtr> lookupAsync(const std::string&);
 
-    Future<Result, LookupDataResultPtr> getPartitionMetadataAsync(const 
DestinationNamePtr&);
+    Future<Result, LookupDataResultPtr> getPartitionMetadataAsync(const 
TopicNamePtr&);
 };
 }  // namespace pulsar
 
diff --git a/pulsar-client-cpp/lib/LookupService.h 
b/pulsar-client-cpp/lib/LookupService.h
index 36ba800..48d5595 100644
--- a/pulsar-client-cpp/lib/LookupService.h
+++ b/pulsar-client-cpp/lib/LookupService.h
@@ -22,25 +22,25 @@
 #include <lib/LookupDataResult.h>
 #include <pulsar/Result.h>
 #include <lib/Future.h>
-#include <lib/DestinationName.h>
 #include <lib/LogUtils.h>
+#include <lib/TopicName.h>
 
 namespace pulsar {
 class LookupService {
    public:
     /*
-     * @param    destinationName - topic name
+     * @param    topicName - topic name
      *
-     * Looks up the owner broker for the given destination name
+     * Looks up the owner broker for the given topic name
      */
-    virtual Future<Result, LookupDataResultPtr> lookupAsync(const std::string& 
destinationName) = 0;
+    virtual Future<Result, LookupDataResultPtr> lookupAsync(const std::string& 
topicName) = 0;
 
     /*
-     * @param    dn - pointer to destination (topic) name
+     * @param    topicName - pointer to topic name
      *
      * Gets Partition metadata
      */
-    virtual Future<Result, LookupDataResultPtr> 
getPartitionMetadataAsync(const DestinationNamePtr& dn) = 0;
+    virtual Future<Result, LookupDataResultPtr> 
getPartitionMetadataAsync(const TopicNamePtr& topicName) = 0;
 
     virtual ~LookupService() {}
 };
diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc 
b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
index 3d22de9..f9c02b3 100644
--- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
+++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
@@ -23,12 +23,12 @@ DECLARE_LOG_OBJECT()
 namespace pulsar {
 
 PartitionedConsumerImpl::PartitionedConsumerImpl(ClientImplPtr client, const 
std::string& subscriptionName,
-                                                 const DestinationNamePtr 
destinationName,
+                                                 const TopicNamePtr topicName,
                                                  const unsigned int 
numPartitions,
                                                  const ConsumerConfiguration& 
conf)
     : client_(client),
       subscriptionName_(subscriptionName),
-      destinationName_(destinationName),
+      topicName_(topicName),
       numPartitions_(numPartitions),
       numConsumersCreated_(0),
       conf_(conf),
@@ -37,7 +37,7 @@ 
PartitionedConsumerImpl::PartitionedConsumerImpl(ClientImplPtr client, const std
       messages_(1000),
       listenerExecutor_(client->getListenerExecutorProvider()->get()),
       messageListener_(conf.getMessageListener()),
-      topic_(destinationName->toString()) {
+      topic_(topicName->toString()) {
     std::stringstream consumerStrStream;
     consumerStrStream << "[Partitioned Consumer: " << topic_ << "," << 
subscriptionName << ","
                       << numPartitions << "]";
@@ -96,7 +96,7 @@ Result PartitionedConsumerImpl::receive(Message& msg, int 
timeout) {
 }
 
 void PartitionedConsumerImpl::unsubscribeAsync(ResultCallback callback) {
-    LOG_INFO("[" << destinationName_->toString() << "," << subscriptionName_ 
<< "] Unsubscribing");
+    LOG_INFO("[" << topicName_->toString() << "," << subscriptionName_ << "] 
Unsubscribing");
     // change state to Closing, so that no Ready state operation is permitted 
during unsubscribe
     setState(Closing);
     // do not accept un subscribe until we have subscribe to all of the 
partitions of a topic
@@ -108,7 +108,7 @@ void 
PartitionedConsumerImpl::unsubscribeAsync(ResultCallback callback) {
         for (ConsumerList::const_iterator consumer = consumers_.begin(); 
consumer != consumers_.end();
              consumer++) {
             LOG_DEBUG("Unsubcribing Consumer - " << index << " for 
Subscription - " << subscriptionName_
-                                                 << " for Topic - " << 
destinationName_->toString());
+                                                 << " for Topic - " << 
topicName_->toString());
             
(*consumer)->unsubscribeAsync(boost::bind(&PartitionedConsumerImpl::handleUnsubscribeAsync,
                                                       shared_from_this(), _1, 
index++, callback));
         }
@@ -124,7 +124,7 @@ void PartitionedConsumerImpl::handleUnsubscribeAsync(Result 
result, unsigned int
         // or do we still go ahead and check how many could we close 
successfully?
         LOG_DEBUG("handleUnsubscribeAsync callback received in Failed State 
for consumerIndex - "
                   << consumerIndex << "with Result - " << result << " for 
Subscription - "
-                  << subscriptionName_ << " for Topic - " << 
destinationName_->toString());
+                  << subscriptionName_ << " for Topic - " << 
topicName_->toString());
         return;
     }
     lock.unlock();
@@ -139,7 +139,7 @@ void PartitionedConsumerImpl::handleUnsubscribeAsync(Result 
result, unsigned int
     // this means we have successfully closed this partition consumer and no 
unsubscribe has failed so far
     LOG_INFO("Successfully Unsubscribed Consumer - " << consumerIndex << " for 
Subscription - "
                                                      << subscriptionName_ << " 
for Topic - "
-                                                     << 
destinationName_->toString());
+                                                     << 
topicName_->toString());
     unsubscribedSoFar_++;
     if (unsubscribedSoFar_ == numPartitions_) {
         LOG_DEBUG("Unsubscribed all of the partition consumer for subscription 
- " << subscriptionName_);
@@ -178,7 +178,7 @@ void PartitionedConsumerImpl::start() {
 
     // create consumer on each partition
     for (unsigned int i = 0; i < numPartitions_; i++) {
-        std::string topicPartitionName = 
destinationName_->getTopicPartitionName(i);
+        std::string topicPartitionName = topicName_->getTopicPartitionName(i);
         consumer = boost::make_shared<ConsumerImpl>(client_, 
topicPartitionName, subscriptionName_, config,
                                                     internalListenerExecutor, 
Partitioned);
         consumer->getConsumerCreatedFuture().addListener(boost::bind(
@@ -218,8 +218,8 @@ void 
PartitionedConsumerImpl::handleSinglePartitionConsumerCreated(
     assert(partitionIndex < numPartitions_ && partitionIndex >= 0);
     numConsumersCreated_++;
     if (numConsumersCreated_ == numPartitions_) {
-        LOG_INFO("Successfully Subscribed to Partitioned Topic - "
-                 << destinationName_->toString() << " with - " << 
numPartitions_ << " Partitions.");
+        LOG_INFO("Successfully Subscribed to Partitioned Topic - " << 
topicName_->toString() << " with - "
+                                                                   << 
numPartitions_ << " Partitions.");
         state_ = Ready;
         lock.unlock();
         receiveMessages();
diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h 
b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
index 3d1bca1..56d04205 100644
--- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
+++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
@@ -20,7 +20,6 @@
 #define PULSAR_PARTITIONED_CONSUMER_HEADER
 #include "ConsumerImpl.h"
 #include "ClientImpl.h"
-#include "DestinationName.h"
 #include <vector>
 #include <boost/shared_ptr.hpp>
 #include <boost/thread/mutex.hpp>
@@ -29,6 +28,7 @@
 #include "lib/UnAckedMessageTrackerDisabled.h"
 #include <lib/Latch.h>
 #include <lib/PartitionedBrokerConsumerStatsImpl.h>
+#include <lib/TopicName.h>
 
 namespace pulsar {
 class PartitionedConsumerImpl;
@@ -44,7 +44,7 @@ class PartitionedConsumerImpl : public ConsumerImplBase,
         Failed
     };
     PartitionedConsumerImpl(ClientImplPtr client, const std::string& 
subscriptionName,
-                            const DestinationNamePtr destinationName, const 
unsigned int numPartitions,
+                            const TopicNamePtr topicName, const unsigned int 
numPartitions,
                             const ConsumerConfiguration& conf);
     virtual ~PartitionedConsumerImpl();
     virtual Future<Result, ConsumerImplBaseWeakPtr> getConsumerCreatedFuture();
@@ -72,7 +72,7 @@ class PartitionedConsumerImpl : public ConsumerImplBase,
    private:
     const ClientImplPtr client_;
     const std::string subscriptionName_;
-    const DestinationNamePtr destinationName_;
+    const TopicNamePtr topicName_;
     unsigned int numPartitions_;
     unsigned int numConsumersCreated_;
     const ConsumerConfiguration conf_;
diff --git a/pulsar-client-cpp/lib/PartitionedProducerImpl.cc 
b/pulsar-client-cpp/lib/PartitionedProducerImpl.cc
index 2c97ab4..728573a 100644
--- a/pulsar-client-cpp/lib/PartitionedProducerImpl.cc
+++ b/pulsar-client-cpp/lib/PartitionedProducerImpl.cc
@@ -20,11 +20,11 @@
 #include "PartitionedProducerImpl.h"
 #include "LogUtils.h"
 #include <boost/bind.hpp>
+#include <lib/TopicName.h>
 #include <sstream>
 #include "RoundRobinMessageRouter.h"
 #include "SinglePartitionMessageRouter.h"
 #include "TopicMetadataImpl.h"
-#include "DestinationName.h"
 #include "MessageImpl.h"
 
 DECLARE_LOG_OBJECT()
@@ -33,13 +33,12 @@ namespace pulsar {
 
 const std::string PartitionedProducerImpl::PARTITION_NAME_SUFFIX = 
"-partition-";
 
-PartitionedProducerImpl::PartitionedProducerImpl(ClientImplPtr client,
-                                                 const DestinationNamePtr 
destinationName,
+PartitionedProducerImpl::PartitionedProducerImpl(ClientImplPtr client, const 
TopicNamePtr topicName,
                                                  const unsigned int 
numPartitions,
                                                  const ProducerConfiguration& 
config)
     : client_(client),
-      destinationName_(destinationName),
-      topic_(destinationName_->toString()),
+      topicName_(topicName),
+      topic_(topicName_->toString()),
       conf_(config),
       state_(Pending),
       topicMetadata_(new TopicMetadataImpl(numPartitions)) {
@@ -76,7 +75,7 @@ void PartitionedProducerImpl::start() {
     boost::shared_ptr<ProducerImpl> producer;
     // create producer per partition
     for (unsigned int i = 0; i < topicMetadata_->getNumPartitions(); i++) {
-        std::string topicPartitionName = 
destinationName_->getTopicPartitionName(i);
+        std::string topicPartitionName = topicName_->getTopicPartitionName(i);
         producer = boost::make_shared<ProducerImpl>(client_, 
topicPartitionName, conf_);
         producer->getProducerCreatedFuture().addListener(boost::bind(
             &PartitionedProducerImpl::handleSinglePartitionProducerCreated, 
shared_from_this(), _1, _2, i));
diff --git a/pulsar-client-cpp/lib/PartitionedProducerImpl.h 
b/pulsar-client-cpp/lib/PartitionedProducerImpl.h
index b06c592..be73ddc 100644
--- a/pulsar-client-cpp/lib/PartitionedProducerImpl.h
+++ b/pulsar-client-cpp/lib/PartitionedProducerImpl.h
@@ -18,13 +18,13 @@
  */
 #include "ProducerImpl.h"
 #include "ClientImpl.h"
-#include "DestinationName.h"
 #include <vector>
 #include <boost/shared_ptr.hpp>
 #include <boost/scoped_ptr.hpp>
 #include <boost/thread/mutex.hpp>
 #include <pulsar/MessageRoutingPolicy.h>
 #include <pulsar/TopicMetadata.h>
+#include <lib/TopicName.h>
 
 namespace pulsar {
 
@@ -43,8 +43,8 @@ class PartitionedProducerImpl : public ProducerImplBase,
 
     typedef boost::unique_lock<boost::mutex> Lock;
 
-    PartitionedProducerImpl(ClientImplPtr ptr, const DestinationNamePtr 
destinationName,
-                            const unsigned int numPartitions, const 
ProducerConfiguration& config);
+    PartitionedProducerImpl(ClientImplPtr ptr, const TopicNamePtr topicName, 
const unsigned int numPartitions,
+                            const ProducerConfiguration& config);
     virtual ~PartitionedProducerImpl();
 
     virtual void sendAsync(const Message& msg, SendCallback callback);
@@ -84,7 +84,7 @@ class PartitionedProducerImpl : public ProducerImplBase,
    private:
     const ClientImplPtr client_;
 
-    const DestinationNamePtr destinationName_;
+    const TopicNamePtr topicName_;
     const std::string topic_;
 
     boost::scoped_ptr<TopicMetadata> topicMetadata_;
diff --git a/pulsar-client-cpp/lib/ProducerImpl.cc 
b/pulsar-client-cpp/lib/ProducerImpl.cc
index b2ed33f..7f09873 100644
--- a/pulsar-client-cpp/lib/ProducerImpl.cc
+++ b/pulsar-client-cpp/lib/ProducerImpl.cc
@@ -21,10 +21,10 @@
 #include "MessageImpl.h"
 #include "PulsarApi.pb.h"
 #include "Commands.h"
-#include "DestinationName.h"
 #include "BatchMessageContainer.h"
 #include <boost/bind.hpp>
 #include <boost/date_time/local_time/local_time.hpp>
+#include <lib/TopicName.h>
 
 namespace pulsar {
 DECLARE_LOG_OBJECT()
diff --git a/pulsar-client-cpp/lib/DestinationName.cc 
b/pulsar-client-cpp/lib/TopicName.cc
similarity index 60%
rename from pulsar-client-cpp/lib/DestinationName.cc
rename to pulsar-client-cpp/lib/TopicName.cc
index 5c8eb3e..1a79a5d 100644
--- a/pulsar-client-cpp/lib/DestinationName.cc
+++ b/pulsar-client-cpp/lib/TopicName.cc
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-#include "DestinationName.h"
 #include "NamedEntity.h"
 #include "LogUtils.h"
 #include "PartitionedProducerImpl.h"
@@ -25,6 +24,7 @@
 #include <boost/algorithm/string.hpp>
 #include <boost/algorithm/string/find.hpp>
 #include <boost/make_shared.hpp>
+#include <lib/TopicName.h>
 #include <vector>
 #include <iostream>
 #include <sstream>
@@ -36,10 +36,10 @@ namespace pulsar {
 
 typedef boost::unique_lock<boost::mutex> Lock;
 // static members
-CURL* DestinationName::curl = NULL;
-boost::mutex DestinationName::curlHandleMutex;
+CURL* TopicName::curl = NULL;
+boost::mutex TopicName::curlHandleMutex;
 
-CURL* DestinationName::getCurlHandle() {
+CURL* TopicName::getCurlHandle() {
     if (curl == NULL) {
         // this handle can not be shared across threads, so had to get here 
everytime
         curl = curl_easy_init();
@@ -47,30 +47,30 @@ CURL* DestinationName::getCurlHandle() {
     return curl;
 }
 //********************************************************************
-DestinationName::DestinationName() {}
+TopicName::TopicName() {}
 
-bool DestinationName::init(const std::string& destinationName) {
-    destination_ = destinationName;
-    if (destinationName.find("://") == std::string::npos) {
-        LOG_ERROR("Destination Name Invalid, domain not present - " << 
destinationName);
+bool TopicName::init(const std::string& topicName) {
+    topicName_ = topicName;
+    if (topicName.find("://") == std::string::npos) {
+        LOG_ERROR("Topic name is not valid, domain not present - " << 
topicName);
         return false;
     }
-    parse(destination_, domain_, property_, cluster_, namespacePortion_, 
localName_);
+    parse(topicName_, domain_, property_, cluster_, namespacePortion_, 
localName_);
     if (localName_.empty()) {
-        LOG_ERROR("Destination Name is not valid, topic name is empty - " << 
destination_);
+        LOG_ERROR("Topic name is not valid, topic name is empty - " << 
topicName_);
         return false;
     }
     namespaceName_ = NamespaceName::get(property_, cluster_, 
namespacePortion_);
     return true;
 }
-void DestinationName::parse(const std::string& destinationName, std::string& 
domain, std::string& property,
-                            std::string& cluster, std::string& 
namespacePortion, std::string& localName) {
-    std::string destinationNameCopy = destinationName;
-    boost::replace_first(destinationNameCopy, "://", "/");
+void TopicName::parse(const std::string& topicName, std::string& domain, 
std::string& property,
+                      std::string& cluster, std::string& namespacePortion, 
std::string& localName) {
+    std::string topicNameCopy = topicName;
+    boost::replace_first(topicNameCopy, "://", "/");
     std::vector<std::string> pathTokens;
-    boost::algorithm::split(pathTokens, destinationNameCopy, 
boost::algorithm::is_any_of("/"));
+    boost::algorithm::split(pathTokens, topicNameCopy, 
boost::algorithm::is_any_of("/"));
     if (pathTokens.size() < 5) {
-        LOG_ERROR("Destination Name Invalid, does not have enough parts - " << 
destinationName);
+        LOG_ERROR("Topic name is not valid, does not have enough parts - " << 
topicName);
         return;
     }
     domain = pathTokens[0];
@@ -80,13 +80,13 @@ void DestinationName::parse(const std::string& 
destinationName, std::string& dom
     size_t slashIndex = -1;
     // find four '/', whatever is left is topic local name
     for (int i = 0; i < 4; i++) {
-        slashIndex = destinationNameCopy.find('/', slashIndex + 1);
+        slashIndex = topicNameCopy.find('/', slashIndex + 1);
     }
     // get index to next char to '/'
     slashIndex++;
-    localName = destinationNameCopy.substr(slashIndex, 
(destinationNameCopy.size() - slashIndex));
+    localName = topicNameCopy.substr(slashIndex, (topicNameCopy.size() - 
slashIndex));
 }
-std::string DestinationName::getEncodedName(const std::string& 
nameBeforeEncoding) {
+std::string TopicName::getEncodedName(const std::string& nameBeforeEncoding) {
     Lock lock(curlHandleMutex);
     std::string nameAfterEncoding;
     if (getCurlHandle()) {
@@ -104,23 +104,23 @@ std::string DestinationName::getEncodedName(const 
std::string& nameBeforeEncodin
     return nameAfterEncoding;
 }
 
-std::string DestinationName::getDomain() { return domain_; }
+std::string TopicName::getDomain() { return domain_; }
 
-std::string DestinationName::getProperty() { return property_; }
+std::string TopicName::getProperty() { return property_; }
 
-std::string DestinationName::getCluster() { return cluster_; }
+std::string TopicName::getCluster() { return cluster_; }
 
-std::string DestinationName::getNamespacePortion() { return namespacePortion_; 
}
+std::string TopicName::getNamespacePortion() { return namespacePortion_; }
 
-std::string DestinationName::getLocalName() { return localName_; }
+std::string TopicName::getLocalName() { return localName_; }
 
-std::string DestinationName::getEncodedLocalName() { return 
getEncodedName(localName_); }
+std::string TopicName::getEncodedLocalName() { return 
getEncodedName(localName_); }
 
-bool DestinationName::operator==(const DestinationName& other) {
-    return (this->destination_.compare(other.destination_) == 0);
+bool TopicName::operator==(const TopicName& other) {
+    return (this->topicName_.compare(other.topicName_) == 0);
 }
 
-bool DestinationName::validateDestination() {
+bool TopicName::validate() {
     // check domain matches to "persistent", in future check "memory" when 
server is ready
     if (domain_.compare("persistent") != 0) {
         return false;
@@ -133,22 +133,22 @@ bool DestinationName::validateDestination() {
     }
 }
 
-boost::shared_ptr<DestinationName> DestinationName::get(const std::string& 
destination) {
-    boost::shared_ptr<DestinationName> ptr(new DestinationName());
-    if (!ptr->init(destination)) {
-        LOG_ERROR("Destination Name Initialization failed");
-        return boost::shared_ptr<DestinationName>();
+boost::shared_ptr<TopicName> TopicName::get(const std::string& topicName) {
+    boost::shared_ptr<TopicName> ptr(new TopicName());
+    if (!ptr->init(topicName)) {
+        LOG_ERROR("Topic name initialization failed");
+        return boost::shared_ptr<TopicName>();
     }
-    if (ptr->validateDestination()) {
+    if (ptr->validate()) {
         return ptr;
     } else {
-        LOG_ERROR("Destination Name Validation Failed");
-        return boost::shared_ptr<DestinationName>();
+        LOG_ERROR("Topic name validation Failed");
+        return boost::shared_ptr<TopicName>();
     }
 }
 
 // TODO - for now return empty string if there's any error in format, later 
think about better error handling
-std::string DestinationName::getLookupName() {
+std::string TopicName::getLookupName() {
     std::stringstream ss;
     std::string seperator("/");
     ss << domain_ << seperator << property_ << seperator << cluster_ << 
seperator << namespacePortion_
@@ -156,7 +156,7 @@ std::string DestinationName::getLookupName() {
     return ss.str();
 }
 
-std::string DestinationName::toString() {
+std::string TopicName::toString() {
     std::stringstream ss;
     std::string seperator("/");
     ss << domain_ << "://" << property_ << seperator << cluster_ << seperator 
<< namespacePortion_
@@ -164,7 +164,7 @@ std::string DestinationName::toString() {
     return ss.str();
 }
 
-const std::string DestinationName::getTopicPartitionName(unsigned int 
partition) {
+const std::string TopicName::getTopicPartitionName(unsigned int partition) {
     std::stringstream topicPartitionName;
     // make this topic name as well
     topicPartitionName << toString() << 
PartitionedProducerImpl::PARTITION_NAME_SUFFIX << partition;
diff --git a/pulsar-client-cpp/lib/DestinationName.h 
b/pulsar-client-cpp/lib/TopicName.h
similarity index 76%
rename from pulsar-client-cpp/lib/DestinationName.h
rename to pulsar-client-cpp/lib/TopicName.h
index 251a01a..b8d610c 100644
--- a/pulsar-client-cpp/lib/DestinationName.h
+++ b/pulsar-client-cpp/lib/TopicName.h
@@ -16,8 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-#ifndef _PULSAR_DESTINATION_NAME_HEADER_
-#define _PULSAR_DESTINATION_NAME_HEADER_
+#ifndef _PULSAR_TOPIC_NAME_HEADER_
+#define _PULSAR_TOPIC_NAME_HEADER_
 
 #include "NamespaceName.h"
 #include "ServiceUnitId.h"
@@ -30,9 +30,9 @@
 #pragma GCC visibility push(default)
 
 namespace pulsar {
-class DestinationName : public ServiceUnitId {
+class TopicName : public ServiceUnitId {
    private:
-    std::string destination_;
+    std::string topicName_;
     std::string domain_;
     std::string property_;
     std::string cluster_;
@@ -49,8 +49,8 @@ class DestinationName : public ServiceUnitId {
     std::string getLocalName();
     std::string getEncodedLocalName();
     std::string toString();
-    static boost::shared_ptr<DestinationName> get(const std::string& 
destination);
-    bool operator==(const DestinationName& other);
+    static boost::shared_ptr<TopicName> get(const std::string& topicName);
+    bool operator==(const TopicName& other);
     static std::string getEncodedName(const std::string& nameBeforeEncoding);
     const std::string getTopicPartitionName(unsigned int partition);
 
@@ -58,16 +58,16 @@ class DestinationName : public ServiceUnitId {
     static CURL* getCurlHandle();
     static CURL* curl;
     static boost::mutex curlHandleMutex;
-    static void parse(const std::string& destinationName, std::string& domain, 
std::string& property,
+    static void parse(const std::string& topicName, std::string& domain, 
std::string& property,
                       std::string& cluster, std::string& namespacePortion, 
std::string& localName);
-    DestinationName();
-    bool validateDestination();
-    bool init(const std::string& destinationName);
+    TopicName();
+    bool validate();
+    bool init(const std::string& topicName);
 };
-typedef boost::shared_ptr<DestinationName> DestinationNamePtr;
+typedef boost::shared_ptr<TopicName> TopicNamePtr;
 }  // namespace pulsar
 // end of namespace pulsar
 
 #pragma GCC visibility pop
 
-#endif  //_PULSAR_DESTINATION_NAME_HEADER_
+#endif  //_PULSAR_TOPIC_NAME_HEADER_
diff --git a/pulsar-client-cpp/perf/PerfConsumer.cc 
b/pulsar-client-cpp/perf/PerfConsumer.cc
index 97aa8db..337035e 100644
--- a/pulsar-client-cpp/perf/PerfConsumer.cc
+++ b/pulsar-client-cpp/perf/PerfConsumer.cc
@@ -212,7 +212,7 @@ void startPerfConsumer(const Arguments& args) {
 
     latch.wait();
     LOG_INFO(
-            "Start receiving from " << args.numConsumers << " consumers on " 
<< args.numTopics << " destinations");
+            "Start receiving from " << args.numConsumers << " consumers on " 
<< args.numTopics << " topics");
 
     while (true) {
         std::this_thread::sleep_for(seconds(10));
diff --git a/pulsar-client-cpp/python/pulsar.py 
b/pulsar-client-cpp/python/pulsar.py
index 060e611..ac38b25 100644
--- a/pulsar-client-cpp/python/pulsar.py
+++ b/pulsar-client-cpp/python/pulsar.py
@@ -553,13 +553,13 @@ class Producer:
           A dict of application-defined string properties.
         * `partition_key`:
           Sets the partition key for message routing. A hash of this key is 
used
-          to determine the message's destination partition.
+          to determine the message's topic partition.
         * `sequence_id`:
           Specify a custom sequence id for the message being published.
         * `replication_clusters`:
           Override namespace replication clusters. Note that it is the caller's
           responsibility to provide valid cluster names and that all clusters
-          have been previously configured as destinations. Given an empty list,
+          have been previously configured as topics. Given an empty list,
           the message will replicate according to the namespace configuration.
         * `disable_replication`:
           Do not replicate this message.
@@ -603,12 +603,12 @@ class Producer:
           A dict of application0-defined string properties.
         * `partition_key`:
           Sets the partition key for the message routing. A hash of this key is
-          used to determine the message's destination partition.
+          used to determine the message's topic partition.
         * `sequence_id`:
           Specify a custom sequence id for the message being published.
         * `replication_clusters`: Override namespace replication clusters. Note
           that it is the caller's responsibility to provide valid cluster names
-          and that all clusters have been previously configured as 
destinations.
+          and that all clusters have been previously configured as topics.
           Given an empty list, the message will replicate per the namespace
           configuration.
         * `disable_replication`:
diff --git a/pulsar-client-cpp/tests/BasicEndToEndTest.cc 
b/pulsar-client-cpp/tests/BasicEndToEndTest.cc
index 0c3f2b4..dfbf44e 100644
--- a/pulsar-client-cpp/tests/BasicEndToEndTest.cc
+++ b/pulsar-client-cpp/tests/BasicEndToEndTest.cc
@@ -21,13 +21,13 @@
 #include <boost/lexical_cast.hpp>
 #include <lib/LogUtils.h>
 #include <pulsar/MessageBuilder.h>
-#include "DestinationName.h"
 #include <lib/Commands.h>
 #include <sstream>
 #include "boost/date_time/posix_time/posix_time.hpp"
 #include "CustomRoutingPolicy.h"
 #include <boost/thread.hpp>
 #include <boost/thread/mutex.hpp>
+#include <lib/TopicName.h>
 #include "PulsarFriend.h"
 #include "HttpHelper.h"
 #include <set>
diff --git a/pulsar-client-cpp/tests/BatchMessageTest.cc 
b/pulsar-client-cpp/tests/BatchMessageTest.cc
index 7ba682f..dda8f79 100644
--- a/pulsar-client-cpp/tests/BatchMessageTest.cc
+++ b/pulsar-client-cpp/tests/BatchMessageTest.cc
@@ -21,8 +21,8 @@
 #include <boost/lexical_cast.hpp>
 #include <lib/LogUtils.h>
 #include <pulsar/MessageBuilder.h>
-#include "DestinationName.h"
 #include <lib/Commands.h>
+#include <lib/TopicName.h>
 #include <sstream>
 #include "boost/date_time/posix_time/posix_time.hpp"
 #include "CustomRoutingPolicy.h"
diff --git a/pulsar-client-cpp/tests/BinaryLookupServiceTest.cc 
b/pulsar-client-cpp/tests/BinaryLookupServiceTest.cc
index b5baeb0..67e6af5 100644
--- a/pulsar-client-cpp/tests/BinaryLookupServiceTest.cc
+++ b/pulsar-client-cpp/tests/BinaryLookupServiceTest.cc
@@ -39,17 +39,17 @@ TEST(BinaryLookupServiceTest, basicLookup) {
     ConnectionPool pool_(conf, ioExecutorProvider_, authData, true);
     BinaryProtoLookupService lookupService(pool_, url);
 
-    std::string topic = "persistent://prop/unit/ns1/destination";
-    DestinationNamePtr dn = DestinationName::get(topic);
+    std::string topic = "persistent://prop/unit/ns1/topic";
+    TopicNamePtr topicName = TopicName::get(topic);
 
-    Future<Result, LookupDataResultPtr> partitionFuture = 
lookupService.getPartitionMetadataAsync(dn);
+    Future<Result, LookupDataResultPtr> partitionFuture = 
lookupService.getPartitionMetadataAsync(topicName);
     LookupDataResultPtr lookupData;
     Result result = partitionFuture.get(lookupData);
     ASSERT_TRUE(lookupData != NULL);
     ASSERT_EQ(0, lookupData->getPartitions());
 
     Future<Result, LookupDataResultPtr> future =
-        lookupService.lookupAsync("persistent://prop/unit/ns1/destination");
+        lookupService.lookupAsync("persistent://prop/unit/ns1/topic");
     result = future.get(lookupData);
 
     ASSERT_EQ(ResultOk, result);
diff --git a/pulsar-client-cpp/tests/ConsumerStatsTest.cc 
b/pulsar-client-cpp/tests/ConsumerStatsTest.cc
index 292c47e..09c4b75 100644
--- a/pulsar-client-cpp/tests/ConsumerStatsTest.cc
+++ b/pulsar-client-cpp/tests/ConsumerStatsTest.cc
@@ -20,7 +20,6 @@
 #include <pulsar/Client.h>
 #include <boost/lexical_cast.hpp>
 #include <lib/LogUtils.h>
-#include <lib/DestinationName.h>
 #include <lib/Commands.h>
 #include "boost/date_time/posix_time/posix_time.hpp"
 #include "CustomRoutingPolicy.h"
@@ -32,6 +31,7 @@
 #include "HttpHelper.h"
 #include <lib/Latch.h>
 #include <lib/PartitionedConsumerImpl.h>
+#include <lib/TopicName.h>
 DECLARE_LOG_OBJECT();
 
 using namespace pulsar;
diff --git a/pulsar-client-cpp/tests/DestinationNameTest.cc 
b/pulsar-client-cpp/tests/DestinationNameTest.cc
deleted file mode 100644
index b88a1d8..0000000
--- a/pulsar-client-cpp/tests/DestinationNameTest.cc
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-#include <DestinationName.h>
-
-#include <gtest/gtest.h>
-
-using namespace pulsar;
-
-TEST(DestinationNameTest, testLookup) {
-    boost::shared_ptr<DestinationName> dn =
-        DestinationName::get("persistent://pulsar/bf1/TESTNS.0/curveballapps");
-    std::string lookup_name = dn->getLookupName();
-    ASSERT_EQ(lookup_name, "persistent/pulsar/bf1/TESTNS.0/curveballapps");
-}
-
-TEST(DestinationNameTest, testDestinationName) {
-    // Compare getters and setters
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent://property/cluster/namespace/destination");
-    ASSERT_EQ("property", dn->getProperty());
-    ASSERT_EQ("cluster", dn->getCluster());
-    ASSERT_EQ("namespace", dn->getNamespacePortion());
-    ASSERT_EQ("persistent", dn->getDomain());
-    ASSERT_EQ(DestinationName::getEncodedName("destination"), 
dn->getLocalName());
-
-    // Compare == operator
-    boost::shared_ptr<DestinationName> dn1 = 
DestinationName::get("persistent://p/c/n/d");
-    boost::shared_ptr<DestinationName> dn2 = 
DestinationName::get("persistent://p/c/n/d");
-    ASSERT_TRUE(*dn1 == *dn2);
-}
-
-TEST(DestinationNameTest, testDestinationNameWithSlashes) {
-    // Compare getters and setters
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent://property/cluster/namespace/destination/name/with/slash");
-    ASSERT_EQ("property", dn->getProperty());
-    ASSERT_EQ("cluster", dn->getCluster());
-    ASSERT_EQ("namespace", dn->getNamespacePortion());
-    ASSERT_EQ("persistent", dn->getDomain());
-    ASSERT_EQ("destination/name/with/slash", dn->getLocalName());
-
-    dn = 
DestinationName::get("persistent://property/cluster/namespace/destination/ends/with/slash/");
-    ASSERT_TRUE(dn != NULL);
-    ASSERT_EQ(DestinationName::getEncodedName("destination/ends/with/slash/"), 
dn->getEncodedLocalName());
-
-    dn = 
DestinationName::get("persistent://property/cluster/namespace/`~!@#$%^&*()-_+=[]{}|\\;:'\"<>,./?");
-    ASSERT_TRUE(dn != NULL);
-    
ASSERT_EQ(DestinationName::getEncodedName("`~!@#$%^&*()-_+=[]{}|\\;:'\"<>,./?"),
-              dn->getEncodedLocalName());
-
-    dn = 
DestinationName::get("persistent://property/cluster/namespace/topic@%*)(&!%$#@#$><?");
-    ASSERT_TRUE(dn != NULL);
-    ASSERT_EQ(DestinationName::getEncodedName("topic@%*)(&!%$#@#$><?"), 
dn->getEncodedLocalName());
-
-    dn = 
DestinationName::get("persistent://property/cluster/namespace/destination//with//double//slash//");
-    ASSERT_TRUE(dn != NULL);
-    
ASSERT_EQ(DestinationName::getEncodedName("destination//with//double//slash//"),
-              dn->getEncodedLocalName());
-
-    dn = 
DestinationName::get("persistent://property/cluster/namespace//destination/starts/with/slash/");
-    ASSERT_TRUE(dn != NULL);
-    
ASSERT_EQ(DestinationName::getEncodedName("/destination/starts/with/slash/"), 
dn->getEncodedLocalName());
-}
-TEST(DestinationNameTest, testEmptyClusterName) {
-    // Compare getters and setters
-    boost::shared_ptr<DestinationName> dn =
-        DestinationName::get("persistent://property//namespace/destination");
-
-    ASSERT_FALSE(dn);
-}
-
-TEST(DestinationNameTest, testExtraSlashes) {
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent://property/cluster//namespace/destination");
-    ASSERT_FALSE(dn);
-    dn = 
DestinationName::get("persistent://property//cluster//namespace//destination");
-    ASSERT_FALSE(dn);
-}
-
-TEST(DestinationNameTest, testIllegalCharacters) {
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent://prop!!!erty/cluster&)&Name/name%%%space/destination");
-    ASSERT_FALSE(dn);
-}
-
-TEST(DestinationNameTest, testIllegalUrl) {
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent:::/property/cluster/namespace/destination");
-    ASSERT_FALSE(dn);
-}
-
-TEST(DestinationNameTest, testEmptyString) {
-    boost::shared_ptr<DestinationName> dn = DestinationName::get("");
-    ASSERT_FALSE(dn);
-}
-
-TEST(DestinationNameTest, testExtraArguments) {
-    boost::shared_ptr<DestinationName> dn =
-        
DestinationName::get("persistent:::/property/cluster/namespace/destination/some/extra/args");
-    ASSERT_FALSE(dn);
-}
diff --git a/pulsar-client-cpp/tests/TopicNameTest.cc 
b/pulsar-client-cpp/tests/TopicNameTest.cc
new file mode 100644
index 0000000..373bbef
--- /dev/null
+++ b/pulsar-client-cpp/tests/TopicNameTest.cc
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+#include <gtest/gtest.h>
+#include <lib/TopicName.h>
+
+using namespace pulsar;
+
+TEST(TopicNameTest, testLookup) {
+    boost::shared_ptr<TopicName> topicName = 
TopicName::get("persistent://pulsar/bf1/TESTNS.0/curveballapps");
+    std::string lookup_name = topicName->getLookupName();
+    ASSERT_EQ(lookup_name, "persistent/pulsar/bf1/TESTNS.0/curveballapps");
+}
+
+TEST(TopicNameTest, testTopicName) {
+    // Compare getters and setters
+    boost::shared_ptr<TopicName> topicName = 
TopicName::get("persistent://property/cluster/namespace/topic");
+    ASSERT_EQ("property", topicName->getProperty());
+    ASSERT_EQ("cluster", topicName->getCluster());
+    ASSERT_EQ("namespace", topicName->getNamespacePortion());
+    ASSERT_EQ("persistent", topicName->getDomain());
+    ASSERT_EQ(TopicName::getEncodedName("topic"), topicName->getLocalName());
+
+    // Compare == operator
+    boost::shared_ptr<TopicName> topicName1 = 
TopicName::get("persistent://p/c/n/d");
+    boost::shared_ptr<TopicName> topicName2 = 
TopicName::get("persistent://p/c/n/d");
+    ASSERT_TRUE(*topicName1 == *topicName2);
+}
+
+TEST(TopicNameTest, testTopicNameWithSlashes) {
+    // Compare getters and setters
+    boost::shared_ptr<TopicName> topicName =
+        
TopicName::get("persistent://property/cluster/namespace/topic/name/with/slash");
+    ASSERT_EQ("property", topicName->getProperty());
+    ASSERT_EQ("cluster", topicName->getCluster());
+    ASSERT_EQ("namespace", topicName->getNamespacePortion());
+    ASSERT_EQ("persistent", topicName->getDomain());
+    ASSERT_EQ("topic/name/with/slash", topicName->getLocalName());
+
+    topicName = 
TopicName::get("persistent://property/cluster/namespace/topic/ends/with/slash/");
+    ASSERT_TRUE(topicName != NULL);
+    ASSERT_EQ(TopicName::getEncodedName("topic/ends/with/slash/"), 
topicName->getEncodedLocalName());
+
+    topicName = 
TopicName::get("persistent://property/cluster/namespace/`~!@#$%^&*()-_+=[]{}|\\;:'\"<>,./?");
+    ASSERT_TRUE(topicName != NULL);
+    ASSERT_EQ(TopicName::getEncodedName("`~!@#$%^&*()-_+=[]{}|\\;:'\"<>,./?"),
+              topicName->getEncodedLocalName());
+
+    topicName = 
TopicName::get("persistent://property/cluster/namespace/topic@%*)(&!%$#@#$><?");
+    ASSERT_TRUE(topicName != NULL);
+    ASSERT_EQ(TopicName::getEncodedName("topic@%*)(&!%$#@#$><?"), 
topicName->getEncodedLocalName());
+
+    topicName = 
TopicName::get("persistent://property/cluster/namespace/topic//with//double//slash//");
+    ASSERT_TRUE(topicName != NULL);
+    ASSERT_EQ(TopicName::getEncodedName("topic//with//double//slash//"), 
topicName->getEncodedLocalName());
+
+    topicName = 
TopicName::get("persistent://property/cluster/namespace//topic/starts/with/slash/");
+    ASSERT_TRUE(topicName != NULL);
+    ASSERT_EQ(TopicName::getEncodedName("/topic/starts/with/slash/"), 
topicName->getEncodedLocalName());
+}
+TEST(TopicNameTest, testEmptyClusterName) {
+    // Compare getters and setters
+    boost::shared_ptr<TopicName> topicName = 
TopicName::get("persistent://property//namespace/topic");
+
+    ASSERT_FALSE(topicName);
+}
+
+TEST(TopicNameTest, testExtraSlashes) {
+    boost::shared_ptr<TopicName> topicName = 
TopicName::get("persistent://property/cluster//namespace/topic");
+    ASSERT_FALSE(topicName);
+    topicName = 
TopicName::get("persistent://property//cluster//namespace//topic");
+    ASSERT_FALSE(topicName);
+}
+
+TEST(TopicNameTest, testIllegalCharacters) {
+    boost::shared_ptr<TopicName> topicName =
+        
TopicName::get("persistent://prop!!!erty/cluster&)&Name/name%%%space/topic");
+    ASSERT_FALSE(topicName);
+}
+
+TEST(TopicNameTest, testIllegalUrl) {
+    boost::shared_ptr<TopicName> topicName = 
TopicName::get("persistent:::/property/cluster/namespace/topic");
+    ASSERT_FALSE(topicName);
+}
+
+TEST(TopicNameTest, testEmptyString) {
+    boost::shared_ptr<TopicName> topicName = TopicName::get("");
+    ASSERT_FALSE(topicName);
+}
+
+TEST(TopicNameTest, testExtraArguments) {
+    boost::shared_ptr<TopicName> topicName =
+        
TopicName::get("persistent:::/property/cluster/namespace/topic/some/extra/args");
+    ASSERT_FALSE(topicName);
+}

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

Reply via email to