You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2012/07/04 15:44:08 UTC

svn commit: r1357264 - in /zookeeper/bookkeeper/trunk: ./ hedwig-client/src/main/cpp/inc/hedwig/ hedwig-client/src/main/cpp/lib/ hedwig-client/src/main/cpp/test/

Author: ivank
Date: Wed Jul  4 13:44:08 2012
New Revision: 1357264

URL: http://svn.apache.org/viewvc?rev=1357264&view=rev
Log:
BOOKKEEPER-320: Let hedwig cpp client could publish messages using Message object instead of string. (jiannan via ivank)

Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/inc/hedwig/publish.h
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.cpp
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.h
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.cpp
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.h
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Wed Jul  4 13:44:08 2012
@@ -34,6 +34,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-274: Hedwig cpp client library should not link to cppunit which is just used for test. (sijie via ivank)
 
+        BOOKKEEPER-320: Let hedwig cpp client could publish messages using Message object instead of string. (jiannan via ivank)
+
       hedwig-server:
 
         BOOKKEEPER-302: No more messages delivered when hub server scans messages over two ledgers. (sijie via ivank)

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/inc/hedwig/publish.h
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/inc/hedwig/publish.h?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/inc/hedwig/publish.h (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/inc/hedwig/publish.h Wed Jul  4 13:44:08 2012
@@ -40,6 +40,8 @@ namespace Hedwig {
     */
     virtual void publish(const std::string& topic, const std::string& message) = 0;
     
+    virtual void publish(const std::string& topic, const Message& message) = 0;
+
     /** 
 	Asynchronously publish message for topic. 
 	
@@ -54,6 +56,8 @@ namespace Hedwig {
     */
     virtual void asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback) = 0;
     
+    virtual void asyncPublish(const std::string& topic, const Message& message, const OperationCallbackPtr& callback) = 0;
+
     virtual ~Publisher() {}
   };
 };

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.cpp
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.cpp?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.cpp (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.cpp Wed Jul  4 13:44:08 2012
@@ -28,12 +28,12 @@ static log4cxx::LoggerPtr logger(log4cxx
 
 using namespace Hedwig;
 
-PubSubDataPtr PubSubData::forPublishRequest(long txnid, const std::string& topic, const std::string& body, const OperationCallbackPtr& callback) {
+PubSubDataPtr PubSubData::forPublishRequest(long txnid, const std::string& topic, const Message& body, const OperationCallbackPtr& callback) {
   PubSubDataPtr ptr(new PubSubData());
   ptr->type = PUBLISH;
   ptr->txnid = txnid;
   ptr->topic = topic;
-  ptr->body = body;
+  ptr->body.CopyFrom(body);
   ptr->callback = callback;
   return ptr;
 }
@@ -87,7 +87,7 @@ const std::string& PubSubData::getTopic(
   return topic;
 }
 
-const std::string& PubSubData::getBody() const {
+const Message& PubSubData::getBody() const {
   return body;
 }
 
@@ -110,7 +110,7 @@ const PubSubRequestPtr PubSubData::getRe
 
     Hedwig::PublishRequest* pubreq = request->mutable_publishrequest();
     Hedwig::Message* msg = pubreq->mutable_msg();
-    msg->set_body(body);
+    msg->CopyFrom(body);
   } else if (type == SUBSCRIBE) {
     LOG4CXX_DEBUG(logger, "Creating subscribe request");
 

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.h
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.h?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.h (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/data.h Wed Jul  4 13:44:08 2012
@@ -61,7 +61,7 @@ namespace Hedwig {
   class PubSubData {
   public:
     // to be used for publish
-    static PubSubDataPtr forPublishRequest(long txnid, const std::string& topic, const std::string& body, const OperationCallbackPtr& callback);
+    static PubSubDataPtr forPublishRequest(long txnid, const std::string& topic, const Message& body, const OperationCallbackPtr& callback);
     static PubSubDataPtr forSubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic,
 					     const OperationCallbackPtr& callback, const SubscriptionOptions& options);
     static PubSubDataPtr forUnsubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const OperationCallbackPtr& callback);
@@ -73,7 +73,7 @@ namespace Hedwig {
     long getTxnId() const;
     const std::string& getSubscriberId() const;
     const std::string& getTopic() const;
-    const std::string& getBody() const;
+    const Message& getBody() const;
     const MessageSeqId getMessageSeqId() const;
 
     void setShouldClaim(bool shouldClaim);
@@ -95,7 +95,7 @@ namespace Hedwig {
     long txnid;
     std::string subscriberid;
     std::string topic;
-    std::string body;
+    Message body;
     bool shouldClaim;
     int messageBound;
     OperationCallbackPtr callback;

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.cpp
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.cpp?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.cpp (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.cpp Wed Jul  4 13:44:08 2012
@@ -44,7 +44,7 @@ PublisherImpl::PublisherImpl(const Clien
   : client(client) {
 }
 
-void PublisherImpl::publish(const std::string& topic, const std::string& message) {
+void PublisherImpl::publish(const std::string& topic, const Message& message) {
   SyncOperationCallback* cb = new SyncOperationCallback(client->getConfiguration().getInt(Configuration::SYNC_REQUEST_TIMEOUT, 
 											  DEFAULT_SYNC_REQUEST_TIMEOUT));
   OperationCallbackPtr callback(cb);
@@ -54,7 +54,13 @@ void PublisherImpl::publish(const std::s
   cb->throwExceptionIfNeeded();  
 }
 
-void PublisherImpl::asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback) {
+void PublisherImpl::publish(const std::string& topic, const std::string& message) {
+  Message msg;
+  msg.set_body(message);
+  publish(topic, msg);
+}
+
+void PublisherImpl::asyncPublish(const std::string& topic, const Message& message, const OperationCallbackPtr& callback) {
   // use release after callback to release the channel after the callback is called
   PubSubDataPtr data = PubSubData::forPublishRequest(client->counter().next(), topic, message, callback);
   
@@ -62,6 +68,12 @@ void PublisherImpl::asyncPublish(const s
   doPublish(channel, data);
 }
 
+void PublisherImpl::asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback) {
+  Message msg;
+  msg.set_body(message);
+  asyncPublish(topic, msg, callback);
+}
+
 void PublisherImpl::doPublish(const DuplexChannelPtr& channel, const PubSubDataPtr& data) {
   channel->storeTransaction(data);
   

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.h
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.h?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.h (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/publisherimpl.h Wed Jul  4 13:44:08 2012
@@ -39,7 +39,10 @@ namespace Hedwig {
     PublisherImpl(const ClientImplPtr& client);
 
     void publish(const std::string& topic, const std::string& message);
+    void publish(const std::string& topic, const Message& message);
+
     void asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback);
+    void asyncPublish(const std::string& topic, const Message& message, const OperationCallbackPtr& callback);
     
     void messageHandler(const PubSubResponsePtr& m, const PubSubDataPtr& txn);
 

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp?rev=1357264&r1=1357263&r2=1357264&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp Wed Jul  4 13:44:08 2012
@@ -42,6 +42,7 @@ private:
   CPPUNIT_TEST_SUITE( PublishTestSuite );
   CPPUNIT_TEST(testSyncPublish);
   CPPUNIT_TEST(testAsyncPublish);
+  CPPUNIT_TEST(testPublishByMessage);
   CPPUNIT_TEST(testMultipleAsyncPublish);
   //  CPPUNIT_TEST(simplePublish);
   //CPPUNIT_TEST(simplePublishAndSubscribe);
@@ -63,6 +64,28 @@ public:
   {
   }
 
+  void testPublishByMessage() {
+    Hedwig::Configuration* conf = new TestServerConfiguration();
+    Hedwig::Client* client = new Hedwig::Client(*conf);
+    Hedwig::Publisher& pub = client->getPublisher();
+
+    Hedwig::Message syncMsg;
+    syncMsg.set_body("sync publish by Message");
+    pub.publish("testTopic", syncMsg);
+
+    SimpleWaitCondition* cond = new SimpleWaitCondition();
+    Hedwig::OperationCallbackPtr testcb(new TestCallback(cond));
+    Hedwig::Message asyncMsg;
+    asyncMsg.set_body("async publish by Message");
+    pub.asyncPublish("testTopic", asyncMsg, testcb);
+    cond->wait();
+    CPPUNIT_ASSERT(cond->wasSuccess());
+    delete cond;
+
+    delete client;
+    delete conf;
+  }
+
   void testSyncPublish() {
     Hedwig::Configuration* conf = new TestServerConfiguration();