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/10/31 18:16:07 UTC

svn commit: r1404255 - in /zookeeper/bookkeeper/trunk: CHANGES.txt hedwig-client/src/main/cpp/lib/clientimpl.cpp hedwig-client/src/main/cpp/lib/clientimpl.h hedwig-client/src/main/cpp/test/publishtest.cpp hedwig-client/src/main/cpp/test/util.h

Author: ivank
Date: Wed Oct 31 17:16:07 2012
New Revision: 1404255

URL: http://svn.apache.org/viewvc?rev=1404255&view=rev
Log:
BOOKKEEPER-434: [Hedwig CPP Client] Delay resolving default host until necessary. (sijie via ivank)

Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.cpp
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.h
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/publishtest.cpp
    zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/util.h

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1404255&r1=1404254&r2=1404255&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Wed Oct 31 17:16:07 2012
@@ -120,6 +120,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-427: TestConcurrentTopicAcquisition hangs every so often (ivank)
 
+        BOOKKEEPER-434: [Hedwig CPP Client] Delay resolving default host until necessary. (sijie 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/lib/clientimpl.cpp
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.cpp?rev=1404255&r1=1404254&r2=1404255&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.cpp (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.cpp Wed Oct 31 17:16:07 2012
@@ -119,15 +119,22 @@ void ResponseHandler::redirectRequest(co
 
   HostAddress h;
   bool redirectToDefaultHost = true;
-  if (response->has_statusmsg()) {
-    try {
-      h = HostAddress::fromString(response->statusmsg());
-      redirectToDefaultHost = false;
-    } catch (std::exception& e) {
+  try {
+    if (response->has_statusmsg()) {
+      try {
+        h = HostAddress::fromString(response->statusmsg());
+        redirectToDefaultHost = false;
+      } catch (std::exception& e) {
+        h = channelManager->getDefaultHost();
+      }
+    } else {
       h = channelManager->getDefaultHost();
     }
-  } else {
-    h = channelManager->getDefaultHost();
+  } catch (std::exception& e) {
+    LOG4CXX_ERROR(logger, "Failed to retrieve redirected host of request " << *data
+                          << " : " << e.what());
+    data->getCallback()->operationFailed(InvalidRedirectException());
+    return;
   }
   if (data->hasTriedServer(h)) {
     LOG4CXX_ERROR(logger, "We've been told to try request [" << data->getTxnId() << "] with [" 
@@ -330,15 +337,15 @@ DuplexChannelManagerPtr DuplexChannelMan
 }
 
 DuplexChannelManager::DuplexChannelManager(const Configuration& conf)
-  : dispatcher(new EventDispatcher(conf)), conf(conf), closed(false), counterobj() {
+  : dispatcher(new EventDispatcher(conf)), conf(conf), closed(false), counterobj(),
+    defaultHostAddress(conf.get(Configuration::DEFAULT_SERVER,
+                                DEFAULT_SERVER_DEFAULT_VAL)) {
   sslEnabled = conf.getBool(Configuration::SSL_ENABLED, DEFAULT_SSL_ENABLED); 
-  defaultHost = HostAddress::fromString(conf.get(Configuration::DEFAULT_SERVER,
-                                                 DEFAULT_SERVER_DEFAULT_VAL));
   if (sslEnabled) {
     sslCtxFactory = SSLContextFactoryPtr(new SSLContextFactory(conf));
   }
   LOG4CXX_DEBUG(logger, "Created DuplexChannelManager " << this << " with default server "
-                        << defaultHost);
+                        << defaultHostAddress);
 }
 
 DuplexChannelManager::~DuplexChannelManager() {
@@ -362,7 +369,13 @@ void DuplexChannelManager::submitOp(cons
   switch (op->getType()) {
   case PUBLISH:
   case UNSUBSCRIBE:
-    channel = getNonSubscriptionChannel(op->getTopic());  
+    try {
+      channel = getNonSubscriptionChannel(op->getTopic());  
+    } catch (std::exception& e) {
+      LOG4CXX_ERROR(logger, "Failed to submit request " << *op << " : " << e.what());
+      op->getCallback()->operationFailed(e);
+      return;
+    }
     break;
   default:
     TopicSubscriber ts(op->getTopic(), op->getSubscriberId());
@@ -418,14 +431,21 @@ void DuplexChannelManager::submitOpThruC
 // Submit a pub/sub request to default server
 void DuplexChannelManager::submitOpToDefaultServer(const PubSubDataPtr& op) {
   DuplexChannelPtr channel;
-  switch (op->getType()) {
-  case PUBLISH:
-  case UNSUBSCRIBE:
-    channel = createNonSubscriptionChannel(defaultHost);
-    break;
-  default:
-    channel = createSubscriptionChannel(defaultHost);
-    break;
+  try {
+    switch (op->getType()) {
+    case PUBLISH:
+    case UNSUBSCRIBE:
+      channel = createNonSubscriptionChannel(getDefaultHost());
+      break;
+    default:
+      channel = createSubscriptionChannel(getDefaultHost());
+      break;
+    }
+  } catch (std::exception& e) {
+    LOG4CXX_ERROR(logger, "Failed to create channel to default host " << defaultHostAddress
+                          << " for request " << op << " : " << e.what());
+    op->getCallback()->operationFailed(e);
+    return;
   }
   OperationCallbackPtr connectCallback(new DefaultServerConnectCallback(shared_from_this(),
                                                                         channel, op));

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.h
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.h?rev=1404255&r1=1404254&r2=1404255&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.h (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/lib/clientimpl.h Wed Oct 31 17:16:07 2012
@@ -270,7 +270,9 @@ namespace Hedwig {
     long nextTxnId();
 
     // return default host
-    inline const HostAddress& getDefaultHost() { return defaultHost; }
+    inline const HostAddress getDefaultHost() {
+      return HostAddress::fromString(defaultHostAddress);
+    }
 
     // set the owner host of a topic
     void setHostForTopic(const std::string& topic, const HostAddress& host);
@@ -402,8 +404,7 @@ namespace Hedwig {
     // counter used for generating transaction ids
     ClientTxnCounter counterobj;
 
-    // default host
-    HostAddress defaultHost;
+    std::string defaultHostAddress;
 
     // non-subscription channels
     std::tr1::unordered_map<HostAddress, DuplexChannelPtr, HostAddressHash > host2channel;

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=1404255&r1=1404254&r2=1404255&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 Oct 31 17:16:07 2012
@@ -156,6 +156,44 @@ TEST(PublishTest, testMultipleAsyncPubli
   delete client;
   delete conf;
 }
+
+class UnresolvedDefaultHostCallback : public Hedwig::OperationCallback {
+public:
+  UnresolvedDefaultHostCallback(SimpleWaitCondition* cond) : cond(cond) {}
+
+  virtual void operationComplete() {
+    cond->setSuccess(false);
+    cond->notify();
+  }
+
+  virtual void operationFailed(const std::exception& exception) {
+    LOG4CXX_ERROR(logger, "Failed with exception : " << exception.what());
+    cond->setSuccess(exception.what() == Hedwig::HostResolutionException().what());
+    cond->notify();
+  }
+
+private:
+  SimpleWaitCondition *cond;
+};
+
+TEST(PublishTest, testPublishWithUnresolvedDefaultHost) {
+  std::string invalidHost("");
+  Hedwig::Configuration* conf = new TestServerConfiguration(invalidHost);
+  
+  SimpleWaitCondition* cond = new SimpleWaitCondition();
+  Hedwig::Client* client = new Hedwig::Client(*conf);
+  Hedwig::Publisher& pub = client->getPublisher();
+  Hedwig::OperationCallbackPtr testcb(new UnresolvedDefaultHostCallback(cond));
+
+  pub.asyncPublish("testTopic", "testPublishWithUnresolvedDefaultHost", testcb);
+  
+  cond->wait();
+  ASSERT_TRUE(cond->wasSuccess());
+  
+  delete cond;
+  delete client;
+  delete conf;
+}
   /*  void simplePublish() {
     LOG4CXX_DEBUG(logger, ">>> simplePublish");
     SimpleWaitCondition* cond = new SimpleWaitCondition();

Modified: zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/util.h
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/util.h?rev=1404255&r1=1404254&r2=1404255&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/util.h (original)
+++ zookeeper/bookkeeper/trunk/hedwig-client/src/main/cpp/test/util.h Wed Oct 31 17:16:07 2012
@@ -142,6 +142,9 @@ public:
   TestServerConfiguration() : address("localhost:4081:9877"),
                               syncTimeout(10000), numThreads(2) {}
 
+  TestServerConfiguration(std::string& defaultServer) :
+    address(defaultServer), syncTimeout(10000), numThreads(2) {}
+
   TestServerConfiguration(int syncTimeout, int numThreads = 2)
     : address("localhost:4081:9877"), syncTimeout(syncTimeout), numThreads(numThreads) {}