Browse Source

ZOOKEEPER-864. Hedwig C++ client improvements

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@1021463 13f79535-47bb-0310-9956-ffa450edef68
Benjamin Reed 14 years ago
parent
commit
b1d9f0a3ca
40 changed files with 2218 additions and 971 deletions
  1. 2 0
      CHANGES.txt
  2. 18 2
      src/contrib/hedwig/client/src/main/cpp/Makefile.am
  3. 25 3
      src/contrib/hedwig/client/src/main/cpp/configure.ac
  4. 18 0
      src/contrib/hedwig/client/src/main/cpp/hedwig-0.1.pc.in
  5. 14 2
      src/contrib/hedwig/client/src/main/cpp/inc/hedwig/client.h
  6. 2 0
      src/contrib/hedwig/client/src/main/cpp/inc/hedwig/exceptions.h
  7. 2 0
      src/contrib/hedwig/client/src/main/cpp/inc/hedwig/publish.h
  8. 2 0
      src/contrib/hedwig/client/src/main/cpp/inc/hedwig/subscribe.h
  9. 25 7
      src/contrib/hedwig/client/src/main/cpp/lib/Makefile.am
  10. 317 309
      src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp
  11. 79 28
      src/contrib/hedwig/client/src/main/cpp/lib/channel.h
  12. 16 4
      src/contrib/hedwig/client/src/main/cpp/lib/client.cpp
  13. 128 246
      src/contrib/hedwig/client/src/main/cpp/lib/clientimpl.cpp
  14. 51 32
      src/contrib/hedwig/client/src/main/cpp/lib/clientimpl.h
  15. 173 0
      src/contrib/hedwig/client/src/main/cpp/lib/data.cpp
  16. 10 6
      src/contrib/hedwig/client/src/main/cpp/lib/data.h
  17. 76 0
      src/contrib/hedwig/client/src/main/cpp/lib/eventdispatcher.cpp
  18. 44 0
      src/contrib/hedwig/client/src/main/cpp/lib/eventdispatcher.h
  19. 9 11
      src/contrib/hedwig/client/src/main/cpp/lib/publisherimpl.cpp
  20. 3 3
      src/contrib/hedwig/client/src/main/cpp/lib/publisherimpl.h
  21. 150 89
      src/contrib/hedwig/client/src/main/cpp/lib/subscriberimpl.cpp
  22. 46 29
      src/contrib/hedwig/client/src/main/cpp/lib/subscriberimpl.h
  23. 0 92
      src/contrib/hedwig/client/src/main/cpp/lib/util.cpp
  24. 11 47
      src/contrib/hedwig/client/src/main/cpp/lib/util.h
  25. 111 0
      src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_asio.m4
  26. 252 0
      src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_base.m4
  27. 149 0
      src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_thread.m4
  28. 49 0
      src/contrib/hedwig/client/src/main/cpp/scripts/log4cpp.conf
  29. 64 0
      src/contrib/hedwig/client/src/main/cpp/scripts/network-delays.sh
  30. 49 0
      src/contrib/hedwig/client/src/main/cpp/scripts/server-control.sh
  31. 95 0
      src/contrib/hedwig/client/src/main/cpp/scripts/tester.sh
  32. 23 3
      src/contrib/hedwig/client/src/main/cpp/test/Makefile.am
  33. 16 3
      src/contrib/hedwig/client/src/main/cpp/test/main.cpp
  34. 29 11
      src/contrib/hedwig/client/src/main/cpp/test/publishtest.cpp
  35. 26 15
      src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp
  36. 9 0
      src/contrib/hedwig/client/src/main/cpp/test/servercontrol.cpp
  37. 4 2
      src/contrib/hedwig/client/src/main/cpp/test/servercontrol.h
  38. 23 7
      src/contrib/hedwig/client/src/main/cpp/test/subscribetest.cpp
  39. 82 15
      src/contrib/hedwig/client/src/main/cpp/test/util.h
  40. 16 5
      src/contrib/hedwig/server/src/test/java/org/apache/hedwig/ServerControlDaemon.java

+ 2 - 0
CHANGES.txt

@@ -144,6 +144,8 @@ IMPROVEMENTS:
   ZOOKEEPER-853. Make zookeeper.is_unrecoverable return True or False
   in zkpython (Andrei Savu via henryr)
 
+  ZOOKEEPER-864. Hedwig C++ client improvements (Ivan Kelly via breed)
+
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)

+ 18 - 2
src/contrib/hedwig/client/src/main/cpp/Makefile.am

@@ -1,3 +1,21 @@
+#
+# 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.
+#
+
 ACLOCAL_AMFLAGS = -I m4
 
 SUBDIRS = lib test
@@ -8,6 +26,4 @@ library_include_HEADERS = inc/hedwig/callback.h inc/hedwig/client.h inc/hedwig/e
 pkgconfigdir = $(libdir)/pkgconfig
 nodist_pkgconfig_DATA = hedwig-0.1.pc
 
-include aminclude.am
-
 EXTRA_DIST = $(DX_CONFIG) doc/html

+ 25 - 3
src/contrib/hedwig/client/src/main/cpp/configure.ac

@@ -1,18 +1,40 @@
+#
+# 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.
+#
+
 AC_INIT([Hedwig C++ Client], [0.1], [zookeeper-dev@hadoop.apache.org], [hedwig-cpp], [http://hadoop.apache.org/zookeeper//])
 
 AC_PREREQ([2.59])
-AM_INIT_AUTOMAKE([1.10 no-define foreign])
+AM_INIT_AUTOMAKE([1.9 no-define foreign])
 AC_CONFIG_HEADERS([config.h])
 AC_PROG_CXX
+AC_LANG([C++])
 AC_CONFIG_FILES([Makefile lib/Makefile test/Makefile hedwig-0.1.pc])
 AC_PROG_LIBTOOL
 AC_CONFIG_MACRO_DIR([m4])
-PKG_CHECK_MODULES([DEPS], [log4cpp >= 0.23 protobuf >= 2.3.0 cppunit])
+PKG_CHECK_MODULES([DEPS], [log4cpp protobuf cppunit])
+AX_BOOST_BASE
+AX_BOOST_ASIO	  
+AX_BOOST_THREAD
 
 DX_HTML_FEATURE(ON)
 DX_INIT_DOXYGEN(hedwig-c++, c-doc.Doxyfile, doc)
 
-CXXFLAGS="$CXXFLAGS -fno-inline"
+CXXFLAGS="$CXXFLAGS -Wall"
 
 AC_OUTPUT
 

+ 18 - 0
src/contrib/hedwig/client/src/main/cpp/hedwig-0.1.pc.in

@@ -1,3 +1,21 @@
+#
+# 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.
+#
+
 prefix=@prefix@
 exec_prefix=@exec_prefix@
 libdir=@libdir@

+ 14 - 2
src/contrib/hedwig/client/src/main/cpp/inc/hedwig/client.h

@@ -25,17 +25,29 @@
 #include <hedwig/publish.h>
 #include <hedwig/exceptions.h>
 #include <boost/noncopyable.hpp>
+#include <boost/shared_ptr.hpp>
 
 namespace Hedwig {
 
   class ClientImpl;
-  typedef std::tr1::shared_ptr<ClientImpl> ClientImplPtr;
+  typedef boost::shared_ptr<ClientImpl> ClientImplPtr;
 
   class Configuration {
+  public:
+    static const std::string DEFAULT_SERVER;
+    static const std::string MESSAGE_CONSUME_RETRY_WAIT_TIME;
+    static const std::string SUBSCRIBER_CONSUME_RETRY_WAIT_TIME;
+    static const std::string MAX_MESSAGE_QUEUE_SIZE;
+    static const std::string RECONNECT_SUBSCRIBE_RETRY_WAIT_TIME;
+    static const std::string SYNC_REQUEST_TIMEOUT;
+
   public:
     Configuration() {};
+    virtual int getInt(const std::string& key, int defaultVal) const = 0;
+    virtual const std::string get(const std::string& key, const std::string& defaultVal) const = 0;
+    virtual bool getBool(const std::string& key, bool defaultVal) const = 0;
 
-    virtual const std::string& getDefaultServer() const;    
+    virtual ~Configuration() {}
   };
 
   /** 

+ 2 - 0
src/contrib/hedwig/client/src/main/cpp/inc/hedwig/exceptions.h

@@ -24,6 +24,8 @@ namespace Hedwig {
 
   class ClientException : public std::exception { };
 
+  class ClientTimeoutException : public ClientException {};
+
   class ServiceDownException : public ClientException {};
   class CannotConnectException : public ClientException {};
   class UnexpectedResponseException : public ClientException {};

+ 2 - 0
src/contrib/hedwig/client/src/main/cpp/inc/hedwig/publish.h

@@ -53,6 +53,8 @@ namespace Hedwig {
 	@param callback Callback which will be used to report success or failure. Success is only reported once the server replies with an ACK response to the publication.
     */
     virtual void asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback) = 0;
+    
+    virtual ~Publisher() {}
   };
 };
 

+ 2 - 0
src/contrib/hedwig/client/src/main/cpp/inc/hedwig/subscribe.h

@@ -44,6 +44,8 @@ namespace Hedwig {
     virtual void stopDelivery(const std::string& topic, const std::string& subscriberId) = 0;
 
     virtual void closeSubscription(const std::string& topic, const std::string& subscriberId) = 0;
+
+    virtual ~Subscriber() {}
   };
 };
 

+ 25 - 7
src/contrib/hedwig/client/src/main/cpp/lib/Makefile.am

@@ -1,14 +1,32 @@
+#
+# 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.
+#
+
 PROTODEF = ../../../../../protocol/src/main/protobuf/PubSubProtocol.proto
 
 lib_LTLIBRARIES = libhedwig01.la
-libhedwig01_la_SOURCES = protocol.cpp channel.cpp client.cpp util.cpp exceptions.cpp clientimpl.cpp publisherimpl.cpp subscriberimpl.cpp
-libhedwig01_la_CPPFLAGS = -I../inc $(DEPS_CFLAGS)
-libhedwig01_la_LIBADD = $(DEPS_LIBS) 
-libhedwig01_la_LDFLAGS = -no-undefined
+libhedwig01_la_SOURCES = protocol.cpp channel.cpp client.cpp util.cpp clientimpl.cpp publisherimpl.cpp subscriberimpl.cpp eventdispatcher.cpp data.cpp
+libhedwig01_la_CPPFLAGS = -I$(top_srcdir)/inc $(DEPS_CFLAGS)
+libhedwig01_la_LIBADD = $(DEPS_LIBS) $(BOOST_CPPFLAGS) 
+libhedwig01_la_LDFLAGS = -no-undefined $(BOOST_ASIO_LIB) $(BOOST_LDFLAGS) $(BOOST_THREAD_LIB)
 
 protocol.cpp: $(PROTODEF)
 	protoc --cpp_out=. -I`dirname $(PROTODEF)` $(PROTODEF)
-	mv PubSubProtocol.pb.cc protocol.cpp
-	sed -i "s/PubSubProtocol.pb.h/hedwig\/protocol.h/" protocol.cpp
-	mv PubSubProtocol.pb.h ../inc/hedwig/protocol.h
+	sed "s/PubSubProtocol.pb.h/hedwig\/protocol.h/" PubSubProtocol.pb.cc > protocol.cpp
+	rm PubSubProtocol.pb.cc
+	mv PubSubProtocol.pb.h $(top_srcdir)/inc/hedwig/protocol.h
 

+ 317 - 309
src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp

@@ -20,6 +20,8 @@
 #include <netinet/in.h>
 #include <arpa/inet.h>
 #include <netinet/tcp.h>
+#include <poll.h>
+#include <iostream>
 
 #include <stdio.h>
 #include <stdlib.h>
@@ -34,403 +36,409 @@
 #include "clientimpl.h"
 
 #include <log4cpp/Category.hh>
+#include <google/protobuf/io/zero_copy_stream_impl.h>
 
 static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__);
 
-const int MAX_MESSAGE_SIZE = 2*1024*1024; // 2 Meg
-
 using namespace Hedwig;
 
-namespace Hedwig {
+DuplexChannel::DuplexChannel(EventDispatcher& dispatcher, const HostAddress& addr, 
+			     const Configuration& cfg, const ChannelHandlerPtr& handler)
+  : dispatcher(dispatcher), address(addr), handler(handler), 
+    socket(dispatcher.getService()), instream(&in_buf), copy_buf(NULL), copy_buf_length(0),
+    state(UNINITIALISED), receiving(false), sending(false)
+{
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Creating DuplexChannel(" << this << ")";
+  }
+}
+
+/*static*/ void DuplexChannel::connectCallbackHandler(DuplexChannelPtr channel,
+						      const boost::system::error_code& error) {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::connectCallbackHandler error(" << error 
+		      << ") channel(" << channel.get() << ")";
+  }
 
-  class RunnableThread {  
-  public:
-    RunnableThread(DuplexChannel& channel, const ChannelHandlerPtr& handler);
-    virtual ~RunnableThread();
-    virtual void entryPoint() = 0;
-    
-    void run();
-    virtual void kill();
-    
-  protected:
-    DuplexChannel& channel;
-    ChannelHandlerPtr handler;
-    pthread_t thread;
-    pthread_attr_t attr;
-  };
+  if (error) {
+    channel->channelDisconnected(ChannelConnectException());
+    channel->setState(DEAD);
+    return;
+  }
+
+  channel->setState(CONNECTED);
+
+  boost::system::error_code ec;
+  boost::asio::ip::tcp::no_delay option(true);
+
+  channel->socket.set_option(option, ec);
+  if (ec) {
+    channel->channelDisconnected(ChannelSetupException());
+    channel->setState(DEAD);
+    return;
+  } 
   
-  typedef std::pair<const PubSubRequest*, OperationCallbackPtr> RequestPair;
+  channel->startSending();
+  channel->startReceiving();
+}
 
-  class PacketsAvailableCondition : public WaitConditionBase {
-  public:
-    PacketsAvailableCondition(std::deque<RequestPair>& queue) : queue(queue), dead(false) {
-    }
+void DuplexChannel::connect() {  
+  setState(CONNECTING);
 
-    ~PacketsAvailableCondition() { wait(); }
+  boost::asio::ip::tcp::endpoint endp(boost::asio::ip::address_v4(address.ip()), address.port());
+  boost::system::error_code error = boost::asio::error::host_not_found;
 
-    bool isTrue() { return dead || !queue.empty(); }
-    void kill() { dead = true; }
+  socket.async_connect(endp, boost::bind(&DuplexChannel::connectCallbackHandler, 
+					 shared_from_this(), 
+					 boost::asio::placeholders::error)); 
+}
 
-  private:
-    std::deque<RequestPair>& queue;
-    bool dead;
-  };
+/*static*/ void DuplexChannel::messageReadCallbackHandler(DuplexChannelPtr channel, 
+							  std::size_t message_size,
+							  const boost::system::error_code& error, 
+							  std::size_t bytes_transferred) {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::messageReadCallbackHandler " << error << ", " 
+		      << bytes_transferred << " channel(" << channel.get() << ")";
+  }
 
-  class WriteThread : public RunnableThread {
-  public: 
-    WriteThread(DuplexChannel& channel, int socketfd, const ChannelHandlerPtr& handler);
-    
-    void entryPoint();
-    void writeRequest(const PubSubRequest& m, const OperationCallbackPtr& callback);
-    virtual void kill();
+  if (error) {
+    LOG.errorStream() << "Invalid read error (" << error << ") bytes_transferred (" 
+		      << bytes_transferred << ") channel(" << channel.get() << ")";
+    channel->channelDisconnected(ChannelReadException());
+    return;
+  }
 
-    ~WriteThread();
-    
-  private:
-    int socketfd;
-
-    PacketsAvailableCondition packetsAvailableWaitCondition;
-    Mutex queueMutex;
-    std::deque<RequestPair> requestQueue;
-    bool dead;
-  };
+  if (channel->copy_buf_length < message_size) {
+    channel->copy_buf_length = message_size;
+    channel->copy_buf = (char*)realloc(channel->copy_buf, channel->copy_buf_length);
+    if (channel->copy_buf == NULL) {
+      LOG.errorStream() << "Error allocating buffer. channel(" << channel.get() << ")";
+      return;
+    }
+  }
   
-  class ReadThread : public RunnableThread {
-  public:
-    ReadThread(DuplexChannel& channel, int socketfd, const ChannelHandlerPtr& handler);
-    
-    void entryPoint();
-    
-    ~ReadThread();
-    
-  private:    
-    int socketfd;
-  };
+  channel->instream.read(channel->copy_buf, message_size);
+  PubSubResponsePtr response(new PubSubResponse());
+  bool err = response->ParseFromArray(channel->copy_buf, message_size);
+
+
+  if (!err) {
+    LOG.errorStream() << "Error parsing message. channel(" << channel.get() << ")";
+
+    channel->channelDisconnected(ChannelReadException());
+    return;
+  } else if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "channel(" << channel.get() << ") : " << channel->in_buf.size() 
+		      << " bytes left in buffer";
+  }
+
+  ChannelHandlerPtr h;
+  {
+    boost::shared_lock<boost::shared_mutex> lock(channel->destruction_lock);
+    if (channel->handler.get()) {
+      h = channel->handler;
+    }
+  }
+  if (h.get()) {
+    h->messageReceived(channel, response);
+  }
+
+  DuplexChannel::readSize(channel);
 }
 
-DuplexChannel::DuplexChannel(const HostAddress& addr, const Configuration& cfg, const ChannelHandlerPtr& handler)
-  : address(addr), handler(handler), writer(NULL), reader(NULL), socketfd(-1), state(UNINITIALISED), txnid2data_lock()
-{
+/*static*/ void DuplexChannel::sizeReadCallbackHandler(DuplexChannelPtr channel, 
+						       const boost::system::error_code& error, 
+						       std::size_t bytes_transferred) {
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Creating DuplexChannel(" << this << ")";
+    LOG.debugStream() << "DuplexChannel::sizeReadCallbackHandler " << error << ", " 
+		      << bytes_transferred << " channel(" << channel.get() << ")";
+  }
+
+  if (error) {
+    LOG.errorStream() << "Invalid read error (" << error << ") bytes_transferred (" 
+		      << bytes_transferred << ") channel(" << channel.get() << ")";
+    channel->channelDisconnected(ChannelReadException());
+    return;
+  }
+  
+  if (channel->in_buf.size() < sizeof(uint32_t)) {
+    LOG.errorStream() << "Not enough data in stream. Must have been an error reading. " 
+		      << " Closing channel(" << channel.get() << ")";
+    channel->channelDisconnected(ChannelReadException());
+    return;
+  }
+
+  uint32_t size;
+  std::istream is(&channel->in_buf);
+  is.read((char*)&size, sizeof(uint32_t));
+  size = ntohl(size);
+
+  int toread = size - channel->in_buf.size();
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << " size of incoming message " << size << ", currently in buffer " 
+		      << channel->in_buf.size() << " channel(" << channel.get() << ")";
+  }
+  if (toread <= 0) {
+    DuplexChannel::messageReadCallbackHandler(channel, size, error, 0);
+  } else {
+    boost::asio::async_read(channel->socket, channel->in_buf,
+			    boost::asio::transfer_at_least(toread),
+			    boost::bind(&DuplexChannel::messageReadCallbackHandler, 
+					channel, size,
+					boost::asio::placeholders::error, 
+					boost::asio::placeholders::bytes_transferred));
   }
 }
 
-void DuplexChannel::connect() {
+/*static*/ void DuplexChannel::readSize(DuplexChannelPtr channel) {
+  if (!channel->isReceiving()) {
+    return;
+  }
+
+  int toread = sizeof(uint32_t) - channel->in_buf.size();
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "DuplexChannel(" << this << ")::connect " << address.getAddressString();
+    LOG.debugStream() << " size of incoming message " << sizeof(uint32_t) 
+		      << ", currently in buffer " << channel->in_buf.size() 
+		      << " channel(" << channel.get() << ")";
   }
 
+  if (toread < 0) {
+    DuplexChannel::sizeReadCallbackHandler(channel, boost::system::error_code(), 0);
+  } else {
+    //  in_buf_size.prepare(sizeof(uint32_t));
+    boost::asio::async_read(channel->socket, channel->in_buf, 
+			    boost::asio::transfer_at_least(sizeof(uint32_t)),
+			    boost::bind(&DuplexChannel::sizeReadCallbackHandler, 
+					channel, 
+					boost::asio::placeholders::error, 
+					boost::asio::placeholders::bytes_transferred));
+  }
+}
+
+void DuplexChannel::startReceiving() {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::startReceiving channel(" << this << ")";
+  }
 
-  socketfd = socket(PF_INET, SOCK_STREAM, IPPROTO_TCP);
+  boost::lock_guard<boost::mutex> lock(receiving_lock);
+  receiving = true;
   
-  if (-1 == socketfd) {
-    LOG.errorStream() << "DuplexChannel(" << this << ") Unable to create socket";
+  DuplexChannel::readSize(shared_from_this());
+}
 
-    throw CannotCreateSocketException();
+bool DuplexChannel::isReceiving() {
+  return receiving;
+}
+
+void DuplexChannel::stopReceiving() {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::stopReceiving channel(" << this << ")";
   }
 
-  if (-1 == ::connect(socketfd, (const struct sockaddr *)&(address.socketAddress()), sizeof(struct sockaddr_in))) {
-    LOG.errorStream() << "DuplexChannel(" << this << ") Could not connect socket";
-    close(socketfd);
+  boost::lock_guard<boost::mutex> lock(receiving_lock);
+  receiving = false;
+}
 
-    throw CannotConnectException();
+void DuplexChannel::startSending() {
+  {
+    boost::shared_lock<boost::shared_mutex> lock(state_lock);
+    if (state != CONNECTED) {
+      return;
+    }
   }
 
+  boost::lock_guard<boost::mutex> lock(sending_lock);
+  if (sending) {
+    return;
+  }
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::startSending channel(" << this << ")";
+  }
 
-  int flag = 1;
-  int res = 0;
-  if ((res = setsockopt(socketfd, IPPROTO_TCP, TCP_NODELAY, &flag, sizeof(int))) != 0){
-    close(socketfd);
-    LOG.errorStream() << "Error setting nodelay on (" << this << ") " << res;
-    throw ChannelSetupException();
+  WriteRequest w;
+  { 
+    boost::lock_guard<boost::mutex> lock(write_lock);
+    if (write_queue.empty()) {
+      return;
+    }
+    w = write_queue.front();
+    write_queue.pop_front();
   }
 
-  reader = new ReadThread(*this, socketfd, handler);
-  writer = new WriteThread(*this, socketfd, handler);
+  sending = true;
 
-  reader->run();
-  writer->run();
+  std::ostream os(&out_buf);
+  uint32_t size = htonl(w.first->ByteSize());
+  os.write((char*)&size, sizeof(uint32_t));
+  
+  bool err = w.first->SerializeToOstream(&os);
+  if (!err) {
+    w.second->operationFailed(ChannelWriteException());
+    channelDisconnected(ChannelWriteException());
+    return;
+  }
 
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "DuplexChannel(" << this << ")::connect successful. Notifying handler.";
-  }    
-  state = CONNECTED;
-  handler->channelConnected(this);
+  boost::asio::async_write(socket, out_buf, 
+			   boost::bind(&DuplexChannel::writeCallbackHandler, 
+				       shared_from_this(), 
+				       w.second,
+				       boost::asio::placeholders::error, 
+				       boost::asio::placeholders::bytes_transferred));
 }
 
+
 const HostAddress& DuplexChannel::getHostAddress() const {
   return address;
 }
 
+void DuplexChannel::channelDisconnected(const std::exception& e) {
+  setState(DEAD);
+  
+  {
+    boost::lock_guard<boost::mutex> lock(write_lock);
+    while (!write_queue.empty()) {
+      WriteRequest w = write_queue.front();
+      write_queue.pop_front();
+      w.second->operationFailed(e);
+    }
+  }
+
+  ChannelHandlerPtr h;
+  {
+    boost::shared_lock<boost::shared_mutex> lock(destruction_lock);
+    if (handler.get()) {
+      h = handler;
+    }
+  }
+  if (h.get()) {
+    h->channelDisconnected(shared_from_this(), e);
+  }
+}
+
 void DuplexChannel::kill() {
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Killing duplex channel (" << this << ")";
   }    
+  
+  bool connected = false;
+  {
+    boost::shared_lock<boost::shared_mutex> statelock(state_lock);
+    connected = (state == CONNECTING || state == CONNECTED);
+  }
 
-  destruction_lock.lock();
-  if (state == CONNECTED) {
-    state = DEAD;
-    
-    destruction_lock.unlock();
-    
-    if (socketfd != -1) {
-      shutdown(socketfd, SHUT_RDWR);
-    }
+  boost::lock_guard<boost::shared_mutex> lock(destruction_lock);
+  if (connected) {
+    setState(DEAD);
     
-    if (writer) {
-      writer->kill();
-      delete writer;
-    }
-    if (reader) {
-      reader->kill();
-      delete reader;
-    }
-    if (socketfd != -1) {
-      close(socketfd);
-    }
-  } else {
-    destruction_lock.unlock();
+    socket.cancel();
+    socket.shutdown(boost::asio::ip::tcp::socket::shutdown_both);
+    socket.close();
   }
-  handler = ChannelHandlerPtr(); // clear the handler in case it ever referenced the channel
+  handler = ChannelHandlerPtr(); // clear the handler in case it ever referenced the channel*/
 }
 
 DuplexChannel::~DuplexChannel() {
   /** If we are going away, fail all transactions that haven't been completed */
   failAllTransactions();
   kill();
-
+  free(copy_buf);
+  copy_buf = NULL;
+  copy_buf_length = 0;
 
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Destroying DuplexChannel(" << this << ")";
-  }    
+  }
 }
 
-void DuplexChannel::writeRequest(const PubSubRequest& m, const OperationCallbackPtr& callback) {
-  if (state != CONNECTED) {
-    LOG.errorStream() << "Tried to write transaction [" << m.txnid() << "] to a channel [" << this << "] which is " << (state == DEAD ? "DEAD" : "UNINITIALISED");
-    callback->operationFailed(UninitialisedChannelException());
+/*static*/ void DuplexChannel::writeCallbackHandler(DuplexChannelPtr channel, OperationCallbackPtr callback,
+						    const boost::system::error_code& error, 
+						    std::size_t bytes_transferred) {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "DuplexChannel::writeCallbackHandler " << error << ", " 
+		      << bytes_transferred << " channel(" << channel.get() << ")";
   }
-			      
-  writer->writeRequest(m, callback);
-}
 
-/**
-   Store the transaction data for a request.
-*/
-void DuplexChannel::storeTransaction(const PubSubDataPtr& data) {
-  txnid2data_lock.lock();
-  txnid2data[data->getTxnId()] = data;
-  txnid2data_lock.unlock();;
-}
+  if (error) {
+    callback->operationFailed(ChannelWriteException());
+    channel->channelDisconnected(ChannelWriteException());
+    return;
+  }
 
-/**
-   Give the transaction back to the caller. 
-*/
-PubSubDataPtr DuplexChannel::retrieveTransaction(long txnid) {
-  txnid2data_lock.lock();
-  PubSubDataPtr data = txnid2data[txnid];
-  txnid2data.erase(txnid);
-  txnid2data_lock.unlock();
-  return data;
-}
+  callback->operationComplete();
 
-void DuplexChannel::failAllTransactions() {
-  txnid2data_lock.lock();
-  for (TransactionMap::iterator iter = txnid2data.begin(); iter != txnid2data.end(); ++iter) {
-    PubSubDataPtr& data = (*iter).second;
-    data->getCallback()->operationFailed(ChannelDiedException());
-  }
-  txnid2data.clear();
-  txnid2data_lock.unlock();
-}
+  channel->out_buf.consume(bytes_transferred);
 
-/** 
-Entry point for pthread initialisation
-*/
-void* ThreadEntryPoint(void *obj) {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Thread entered (" << obj << ")";
+  {
+    boost::lock_guard<boost::mutex> lock(channel->sending_lock);
+    channel->sending = false;
   }
 
-  RunnableThread* thread = (RunnableThread*) obj;
-  thread->entryPoint();
+  channel->startSending();
 }
- 
-RunnableThread::RunnableThread(DuplexChannel& channel, const ChannelHandlerPtr& handler) 
-  : channel(channel), handler(handler)
-{
-  //  pthread_cond_init(&deathlock, NULL);
-}
-
-void RunnableThread::run() {
-  int ret;
 
+void DuplexChannel::writeRequest(const PubSubRequestPtr& m, const OperationCallbackPtr& callback) {
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Running thread (" << this << ")";
-  }    
-  
-  pthread_attr_init(&attr);
-  ret = pthread_create(&thread, &attr, ThreadEntryPoint, this);
-  if (ret != 0) {
-    LOG.errorStream() << "Error creating thread (" << this << "). Notifying handler.";
-    handler->exceptionOccurred(&channel, ChannelThreadException());
+    LOG.debugStream() << "DuplexChannel::writeRequest channel(" << this << ") txnid(" 
+		      << m->txnid() << ") shouldClaim("<< m->has_shouldclaim() << ", " 
+		      << m->shouldclaim() << ")";
   }
-}
 
-void RunnableThread::kill() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Killing thread (" << this << ")";
-  }    
+  {
+    boost::shared_lock<boost::shared_mutex> lock(state_lock);
+    if (state != CONNECTED && state != CONNECTING) {
+      LOG.errorStream() << "Tried to write transaction [" << m->txnid() << "] to a channel [" 
+			<< this << "] which is " << (state == DEAD ? "DEAD" : "UNINITIALISED");
+      callback->operationFailed(UninitialisedChannelException());
+    }
+  }
 
-  pthread_cancel(thread);
-  pthread_join(thread, NULL);
+  { 
+    boost::lock_guard<boost::mutex> lock(write_lock);
+    WriteRequest w(m, callback);
+    write_queue.push_back(w);
+  }
 
-  pthread_attr_destroy(&attr);
+  startSending();
 }
 
-RunnableThread::~RunnableThread() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Deleting thread (" << this << ")";
-  }    
-}
 /**
-Writer thread
+   Store the transaction data for a request.
 */
-WriteThread::WriteThread(DuplexChannel& channel, int socketfd, const ChannelHandlerPtr& handler) 
-  : RunnableThread(channel, handler), socketfd(socketfd), packetsAvailableWaitCondition(requestQueue), queueMutex(), dead(false) {
-  
-}
-
-// should probably be using a queue here.
-void WriteThread::writeRequest(const PubSubRequest& m, const OperationCallbackPtr& callback) {
-  #warning "you should validate these inputs"
+void DuplexChannel::storeTransaction(const PubSubDataPtr& data) {
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Adding message to queue " << &m;
-  }
-  packetsAvailableWaitCondition.lock();
-  queueMutex.lock();
-  requestQueue.push_back(RequestPair(&m, callback));
-  queueMutex.unlock();;
-
-  packetsAvailableWaitCondition.signalAndUnlock();
-}
-  
-void WriteThread::entryPoint() {
-  while (true) {
-    packetsAvailableWaitCondition.wait();
-
-    if (dead) {
-      if (LOG.isDebugEnabled()) {
-	LOG.debugStream() << "returning from thread " << this;
-      }
-      return;
-    }
-    while (!requestQueue.empty()) { 
-      queueMutex.lock();;
-      RequestPair currentRequest = requestQueue.front();;
-      requestQueue.pop_front();
-      queueMutex.unlock();
-      if (LOG.isDebugEnabled()) {
-	LOG.debugStream() << "Writing message to socket " << currentRequest.first;
-      }
-      
-      uint32_t size = htonl(currentRequest.first->ByteSize());
-      write(socketfd, &size, sizeof(size));
-      
-      bool res = currentRequest.first->SerializeToFileDescriptor(socketfd);
-      
-      if (!res || errno != 0) {
-	LOG.errorStream() << "Error writing to socket (" << this << ") errno(" << errno << ") res(" << res << "). Disconnected.";
-	ChannelWriteException e;
-	
-	currentRequest.second->operationFailed(e);
-	channel.kill(); // make sure it's dead
-	handler->channelDisconnected(&channel, e);
-	
-	return;
-      } else {
-	currentRequest.second->operationComplete();
-      }
-    }  
+    LOG.debugStream() << "Storing txnid(" << data->getTxnId() << ") for channel(" << this << ")";
   }
-}
-
-void WriteThread::kill() {
-  dead = true;
-  packetsAvailableWaitCondition.lock();
-  packetsAvailableWaitCondition.kill();
-  packetsAvailableWaitCondition.signalAndUnlock();
-  
-  RunnableThread::kill();
-}
-
-WriteThread::~WriteThread() {
-  queueMutex.unlock();
+  boost::lock_guard<boost::mutex> lock(txnid2data_lock);
+  txnid2data[data->getTxnId()] = data;
 }
 
 /**
-Reader Thread
+   Give the transaction back to the caller. 
 */
+PubSubDataPtr DuplexChannel::retrieveTransaction(long txnid) {
+  boost::lock_guard<boost::mutex> lock(txnid2data_lock);
 
-ReadThread::ReadThread(DuplexChannel& channel, int socketfd, const ChannelHandlerPtr& handler) 
-  : RunnableThread(channel, handler), socketfd(socketfd) {
-}
-  
-void ReadThread::entryPoint() {
-  PubSubResponse* response = new PubSubResponse();
-  uint8_t* dataarray = NULL;//(uint8_t*)malloc(MAX_MESSAGE_SIZE); // shouldn't be allocating every time. check that there's a max size
-  int currentbufsize = 0;
-  
-  while (true) {
-    uint32_t size = 0;
-    int bytesread = 0;
-
-    bytesread = read(socketfd, &size, sizeof(size));
-    size = ntohl(size);
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Start reading packet of size: " << size;
-    }
-    if (bytesread < 1 || size > MAX_MESSAGE_SIZE) {
-      LOG.errorStream() << "Zero read from socket or unreasonable size read, size(" << size << ") errno(" << errno << ") " << strerror(errno);
-      channel.kill(); // make sure it's dead
-      handler->channelDisconnected(&channel, ChannelReadException());
-      break;
-    }
-
-    if (currentbufsize < size) {
-      dataarray = (uint8_t*)realloc(dataarray, size);
-    }
-    if (dataarray == NULL) {
-      LOG.errorStream() << "Error allocating input buffer of size " << size << " errno(" << errno << ") " << strerror(errno);
-      channel.kill(); // make sure it's dead
-      handler->channelDisconnected(&channel, ChannelReadException());
-      
-      break;
-    }
-    
-    memset(dataarray, 0, size);
-    bytesread = read(socketfd, dataarray, size);
-    bool res = response->ParseFromArray(dataarray, size);
+  PubSubDataPtr data = txnid2data[txnid];
+  txnid2data.erase(txnid);
+  if (data == NULL) {
+    LOG.errorStream() << "Transaction txnid(" << txnid 
+		      << ") doesn't exist in channel (" << this << ")";
+  }
 
+  return data;
+}
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Packet read ";
-    }
-    
-    if (!res && errno != 0 || bytesread < size) {
-      LOG.errorStream() << "Error reading from socket (" << this << ") errno(" << errno << ") res(" << res << "). Disconnected.";
-      channel.kill(); // make sure it's dead
-      handler->channelDisconnected(&channel, ChannelReadException());
-
-      break;
-    } else {
-      handler->messageReceived(&channel, *response);
-    }
+void DuplexChannel::failAllTransactions() {
+  boost::lock_guard<boost::mutex> lock(txnid2data_lock);
+  for (TransactionMap::iterator iter = txnid2data.begin(); iter != txnid2data.end(); ++iter) {
+    PubSubDataPtr& data = (*iter).second;
+    data->getCallback()->operationFailed(ChannelDiedException());
   }
-  free(dataarray);
-  delete response;
+  txnid2data.clear();
 }
 
-ReadThread::~ReadThread() {
+void DuplexChannel::setState(State s) {
+  boost::lock_guard<boost::shared_mutex> lock(state_lock);
+  state = s;
 }

+ 79 - 28
src/contrib/hedwig/client/src/main/cpp/lib/channel.h

@@ -23,9 +23,20 @@
 #include <hedwig/client.h>
 #include "util.h"
 #include "data.h"
+#include "eventdispatcher.h"
+
 #include <tr1/memory>
 #include <tr1/unordered_map>
 
+#include <google/protobuf/io/zero_copy_stream_impl.h>
+
+#include <boost/shared_ptr.hpp>
+#include <boost/enable_shared_from_this.hpp>
+
+#include <boost/asio/ip/tcp.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/shared_mutex.hpp>
+
 namespace Hedwig {
   class ChannelException : public std::exception { };
   class UninitialisedChannelException : public ChannelException {};
@@ -33,6 +44,7 @@ namespace Hedwig {
   class ChannelConnectException : public ChannelException {};
   class CannotCreateSocketException : public ChannelConnectException {};
   class ChannelSetupException : public ChannelConnectException {};
+  class ChannelNotConnectedException : public ChannelConnectException {};
 
   class ChannelDiedException : public ChannelException {};
 
@@ -40,66 +52,105 @@ namespace Hedwig {
   class ChannelReadException : public ChannelException {};
   class ChannelThreadException : public ChannelException {};
 
+  class DuplexChannel;
+  typedef boost::shared_ptr<DuplexChannel> DuplexChannelPtr;
 
   class ChannelHandler {
   public:
-    virtual void messageReceived(DuplexChannel* channel, const PubSubResponse& m) = 0;
-    virtual void channelConnected(DuplexChannel* channel) = 0;
+    virtual void messageReceived(const DuplexChannelPtr& channel, const PubSubResponsePtr& m) = 0;
+    virtual void channelConnected(const DuplexChannelPtr& channel) = 0;
 
-    virtual void channelDisconnected(DuplexChannel* channel, const std::exception& e) = 0;
-    virtual void exceptionOccurred(DuplexChannel* channel, const std::exception& e) = 0;
+    virtual void channelDisconnected(const DuplexChannelPtr& channel, const std::exception& e) = 0;
+    virtual void exceptionOccurred(const DuplexChannelPtr& channel, const std::exception& e) = 0;
 
     virtual ~ChannelHandler() {}
   };
-  typedef std::tr1::shared_ptr<ChannelHandler> ChannelHandlerPtr;
 
-  class WriteThread;
-  class ReadThread;
+  typedef boost::shared_ptr<ChannelHandler> ChannelHandlerPtr;
 
-  class DuplexChannel {
-  public:
-    DuplexChannel(const HostAddress& addr, const Configuration& cfg, const ChannelHandlerPtr& handler);
 
+  class DuplexChannel : public boost::enable_shared_from_this<DuplexChannel> {
+  public:
+    DuplexChannel(EventDispatcher& dispatcher, const HostAddress& addr, 
+		  const Configuration& cfg, const ChannelHandlerPtr& handler);
+    static void connectCallbackHandler(DuplexChannelPtr channel, 
+				       const boost::system::error_code& error);
     void connect();
 
-    void writeRequest(const PubSubRequest& m, const OperationCallbackPtr& callback);
+    static void writeCallbackHandler(DuplexChannelPtr channel, OperationCallbackPtr callback, 
+				     const boost::system::error_code& error, 
+				     std::size_t bytes_transferred);
+    void writeRequest(const PubSubRequestPtr& m, const OperationCallbackPtr& callback);
     
     const HostAddress& getHostAddress() const;
 
     void storeTransaction(const PubSubDataPtr& data);
     PubSubDataPtr retrieveTransaction(long txnid);
     void failAllTransactions();
+
+    static void sizeReadCallbackHandler(DuplexChannelPtr channel, 
+					const boost::system::error_code& error, 
+					std::size_t bytes_transferred);
+    static void messageReadCallbackHandler(DuplexChannelPtr channel, std::size_t messagesize, 
+					   const boost::system::error_code& error, 
+					   std::size_t bytes_transferred);
+    static void readSize(DuplexChannelPtr channel);
+
+    void startReceiving();
+    bool isReceiving();
+    void stopReceiving();
     
+    void startSending();
+
+    void channelDisconnected(const std::exception& e);
     virtual void kill();
 
-    ~DuplexChannel();
+    virtual ~DuplexChannel();
   private:
+    enum State { UNINITIALISED, CONNECTING, CONNECTED, DEAD };
+
+    void setState(State s);
+
+    EventDispatcher& dispatcher;
+
     HostAddress address;
     ChannelHandlerPtr handler;
-    int socketfd;
-    WriteThread *writer;
-    ReadThread *reader;
+
+    boost::asio::ip::tcp::socket socket;
+    boost::asio::streambuf in_buf;
+    std::istream instream;
     
-    enum State { UNINITIALISED, CONNECTED, DEAD };
+    // only exists because protobufs can't play nice with streams (if there's more than message len in it, it tries to read all)
+    char* copy_buf;
+    std::size_t copy_buf_length;
+
+    boost::asio::streambuf out_buf;
+    
+    typedef std::pair<PubSubRequestPtr, OperationCallbackPtr> WriteRequest;
+    boost::mutex write_lock;
+    std::deque<WriteRequest> write_queue;
+
     State state;
+    boost::shared_mutex state_lock;
+
+    bool receiving;
+    boost::mutex receiving_lock;
     
+    bool sending;
+    boost::mutex sending_lock;
+
     typedef std::tr1::unordered_map<long, PubSubDataPtr> TransactionMap;
+
     TransactionMap txnid2data;
-    Mutex txnid2data_lock;
-    Mutex destruction_lock;
+    boost::mutex txnid2data_lock;
+    boost::shared_mutex destruction_lock;
   };
   
-  typedef std::tr1::shared_ptr<DuplexChannel> DuplexChannelPtr;
-};
 
-namespace std 
-{
-  namespace tr1 
-  {
-  // defined in util.cpp
-  template <> struct hash<Hedwig::DuplexChannelPtr> : public unary_function<Hedwig::DuplexChannelPtr, size_t> {
-    size_t operator()(const Hedwig::DuplexChannelPtr& channel) const;
+  struct DuplexChannelPtrHash : public std::unary_function<DuplexChannelPtr, size_t> {
+    size_t operator()(const Hedwig::DuplexChannelPtr& channel) const {
+      return reinterpret_cast<size_t>(channel.get());
+    }
   };
-  }
 };
 #endif

+ 16 - 4
src/contrib/hedwig/client/src/main/cpp/lib/client.cpp

@@ -20,15 +20,23 @@
 #include <memory>
 
 #include "clientimpl.h"
+#include <log4cpp/Category.hh>
+
+static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__);
 
 using namespace Hedwig;
 
-const std::string DEFAULT_SERVER = "localhost:4080";
-const std::string& Configuration::getDefaultServer() const {
-  return DEFAULT_SERVER;
-}
+const std::string Configuration::DEFAULT_SERVER = "hedwig.cpp.default_server";
+const std::string Configuration::MESSAGE_CONSUME_RETRY_WAIT_TIME = "hedwig.cpp.message_consume_retry_wait_time";
+const std::string Configuration::SUBSCRIBER_CONSUME_RETRY_WAIT_TIME = "hedwig.cpp.subscriber_consume_retry_wait_time";
+const std::string Configuration::MAX_MESSAGE_QUEUE_SIZE = "hedwig.cpp.max_msgqueue_size";
+const std::string Configuration::RECONNECT_SUBSCRIBE_RETRY_WAIT_TIME = "hedwig.cpp.reconnect_subscribe_retry_wait_time";
+const std::string Configuration::SYNC_REQUEST_TIMEOUT = "hedwig.cpp.sync_request_timeout";
 
 Client::Client(const Configuration& conf) {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Client::Client (" << this << ")";
+  }
   clientimpl = ClientImpl::Create( conf );
 }
 
@@ -41,6 +49,10 @@ Publisher& Client::getPublisher() {
 }
 
 Client::~Client() {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Client::~Client (" << this << ")";
+  }
+
   clientimpl->Destroy();
 }
 

+ 128 - 246
src/contrib/hedwig/client/src/main/cpp/lib/clientimpl.cpp

@@ -25,31 +25,54 @@ static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__
 
 using namespace Hedwig;
 
+const std::string DEFAULT_SERVER_DEFAULT_VAL = "";
+
+void SyncOperationCallback::wait() {
+  boost::unique_lock<boost::mutex> lock(mut);
+  while(response==PENDING) {
+    if (cond.timed_wait(lock, boost::posix_time::milliseconds(timeout)) == false) {
+      LOG.errorStream() << "Timeout waiting for operation to complete " << this;
+
+      response = TIMEOUT;
+    }
+  }
+}
 
 void SyncOperationCallback::operationComplete() {
-  lock();
-  response = SUCCESS;
-  signalAndUnlock();
+  if (response == TIMEOUT) {
+    LOG.errorStream() << "operationCompleted successfully after timeout " << this;
+    return;
+  }
+
+  {
+    boost::lock_guard<boost::mutex> lock(mut);
+    response = SUCCESS;
+  }
+  cond.notify_all();
 }
 
 void SyncOperationCallback::operationFailed(const std::exception& exception) {
-  lock();
-  if (typeid(exception) == typeid(ChannelConnectException)) {
-    response = NOCONNECT;
-  } else if (typeid(exception) == typeid(ServiceDownException)) {
-    response = SERVICEDOWN;
-  } else if (typeid(exception) == typeid(AlreadySubscribedException)) {
-    response = ALREADY_SUBSCRIBED;
-  } else if (typeid(exception) == typeid(NotSubscribedException)) {
-    response = NOT_SUBSCRIBED;
-  } else {
-    response = UNKNOWN;
+  if (response == TIMEOUT) {
+    LOG.errorStream() << "operationCompleted unsuccessfully after timeout " << this;
+    return;
   }
-  signalAndUnlock();
-}
+
+  {
+    boost::lock_guard<boost::mutex> lock(mut);
     
-bool SyncOperationCallback::isTrue() {
-  return response != PENDING;
+    if (typeid(exception) == typeid(ChannelConnectException)) {
+      response = NOCONNECT;
+    } else if (typeid(exception) == typeid(ServiceDownException)) {
+      response = SERVICEDOWN;
+    } else if (typeid(exception) == typeid(AlreadySubscribedException)) {
+      response = ALREADY_SUBSCRIBED;
+    } else if (typeid(exception) == typeid(NotSubscribedException)) {
+      response = NOT_SUBSCRIBED;
+    } else {
+      response = UNKNOWN;
+    }
+  }
+  cond.notify_all();
 }
 
 void SyncOperationCallback::throwExceptionIfNeeded() {
@@ -68,34 +91,36 @@ void SyncOperationCallback::throwExceptionIfNeeded() {
   case NOT_SUBSCRIBED:
     throw NotSubscribedException();
     break;
+  case TIMEOUT:
+    throw ClientTimeoutException();
+    break;
   default:
     throw ClientException();
     break;
   }
 }
 
-HedwigClientChannelHandler::HedwigClientChannelHandler(ClientImplPtr& client) 
+HedwigClientChannelHandler::HedwigClientChannelHandler(const ClientImplPtr& client) 
   : client(client){
 }
 
-void HedwigClientChannelHandler::messageReceived(DuplexChannel* channel, const PubSubResponse& m) {
-  LOG.debugStream() << "Message received";
-  if (m.has_message()) {
+void HedwigClientChannelHandler::messageReceived(const DuplexChannelPtr& channel, const PubSubResponsePtr& m) {
+  LOG.debugStream() << "Message received txnid(" << m->txnid() << ") status(" 
+		    << m->statuscode() << ")";
+  if (m->has_message()) {
     LOG.errorStream() << "Subscription response, ignore for now";
     return;
   }
   
-  long txnid = m.txnid();
-  PubSubDataPtr data = channel->retrieveTransaction(m.txnid()); 
+  PubSubDataPtr data = channel->retrieveTransaction(m->txnid()); 
   /* you now have ownership of data, don't leave this funciton without deleting it or 
      palming it off to someone else */
 
   if (data == NULL) {
-    LOG.errorStream() << "Transaction " << m.txnid() << " doesn't exist in channel " << channel;
     return;
   }
 
-  if (m.statuscode() == NOT_RESPONSIBLE_FOR_TOPIC) {
+  if (m->statuscode() == NOT_RESPONSIBLE_FOR_TOPIC) {
     client->redirectRequest(channel, data, m);
     return;
   }
@@ -115,17 +140,17 @@ void HedwigClientChannelHandler::messageReceived(DuplexChannel* channel, const P
 }
 
 
-void HedwigClientChannelHandler::channelConnected(DuplexChannel* channel) {
+void HedwigClientChannelHandler::channelConnected(const DuplexChannelPtr& channel) {
   // do nothing 
 }
 
-void HedwigClientChannelHandler::channelDisconnected(DuplexChannel* channel, const std::exception& e) {
+void HedwigClientChannelHandler::channelDisconnected(const DuplexChannelPtr& channel, const std::exception& e) {
   LOG.errorStream() << "Channel disconnected";
 
   client->channelDied(channel);
 }
 
-void HedwigClientChannelHandler::exceptionOccurred(DuplexChannel* channel, const std::exception& e) {
+void HedwigClientChannelHandler::exceptionOccurred(const DuplexChannelPtr& channel, const std::exception& e) {
   LOG.errorStream() << "Exception occurred" << e.what();
 }
 
@@ -142,185 +167,39 @@ Increment the transaction counter and return the new value.
 @returns the next transaction id
 */
 long ClientTxnCounter::next() {  // would be nice to remove lock from here, look more into it
-  mutex.lock();
-  long next= ++counter; 
-  mutex.unlock();
-  return next;
-}
-
-
-
-PubSubDataPtr PubSubData::forPublishRequest(long txnid, const std::string& topic, const std::string& body, const OperationCallbackPtr& callback) {
-  PubSubDataPtr ptr(new PubSubData());
-  ptr->type = PUBLISH;
-  ptr->txnid = txnid;
-  ptr->topic = topic;
-  ptr->body = body;
-  ptr->callback = callback;
-  return ptr;
-}
-
-PubSubDataPtr PubSubData::forSubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const OperationCallbackPtr& callback, SubscribeRequest::CreateOrAttach mode) {
-  PubSubDataPtr ptr(new PubSubData());
-  ptr->type = SUBSCRIBE;
-  ptr->txnid = txnid;
-  ptr->subscriberid = subscriberid;
-  ptr->topic = topic;
-  ptr->callback = callback;
-  ptr->mode = mode;
-  return ptr;  
-}
-
-PubSubDataPtr PubSubData::forUnsubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const OperationCallbackPtr& callback) {
-  PubSubDataPtr ptr(new PubSubData());
-  ptr->type = UNSUBSCRIBE;
-  ptr->txnid = txnid;
-  ptr->subscriberid = subscriberid;
-  ptr->topic = topic;
-  ptr->callback = callback;
-  return ptr;  
-}
-
-PubSubDataPtr PubSubData::forConsumeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const MessageSeqId msgid) {
-  PubSubDataPtr ptr(new PubSubData());
-  ptr->type = CONSUME;
-  ptr->txnid = txnid;
-  ptr->subscriberid = subscriberid;
-  ptr->topic = topic;
-  ptr->msgid = msgid;
-  return ptr;  
-}
-
-PubSubData::PubSubData() : request(NULL) {  
-}
-
-PubSubData::~PubSubData() {
-  if (request != NULL) {
-    delete request;
-  }
-}
-
-OperationType PubSubData::getType() const {
-  return type;
-}
+  boost::lock_guard<boost::mutex> lock(mutex);
 
-long PubSubData::getTxnId() const {
-  return txnid;
-}
-
-const std::string& PubSubData::getTopic() const {
-  return topic;
-}
-
-const std::string& PubSubData::getBody() const {
-  return body;
-}
-
-const PubSubRequest& PubSubData::getRequest() {
-  if (request != NULL) {
-    delete request;
-    request = NULL;
-  }
-  request = new Hedwig::PubSubRequest();
-  request->set_protocolversion(Hedwig::VERSION_ONE);
-  request->set_type(type);
-  request->set_txnid(txnid);
-  request->set_shouldclaim(shouldClaim);
-  request->set_topic(topic);
-    
-  if (type == PUBLISH) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Creating publish request";
-    }
-    Hedwig::PublishRequest* pubreq = request->mutable_publishrequest();
-    Hedwig::Message* msg = pubreq->mutable_msg();
-    msg->set_body(body);
-  } else if (type == SUBSCRIBE) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Creating subscribe request";
-    }
-
-    Hedwig::SubscribeRequest* subreq = request->mutable_subscriberequest();
-    subreq->set_subscriberid(subscriberid);
-    subreq->set_createorattach(mode);
-  } else if (type == CONSUME) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Creating consume request";
-    }
-
-    Hedwig::ConsumeRequest* conreq = request->mutable_consumerequest();
-    conreq->set_subscriberid(subscriberid);
-    conreq->mutable_msgid()->CopyFrom(msgid);
-  } else if (type == UNSUBSCRIBE) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "Creating unsubscribe request";
-    }
-    
-    Hedwig::UnsubscribeRequest* unsubreq = request->mutable_unsubscriberequest();
-    unsubreq->set_subscriberid(subscriberid);    
-  } else {
-    LOG.errorStream() << "Tried to create a request message for the wrong type [" << type << "]";
-    throw UnknownRequestException();
-  }
-
-
-
-  return *request;
-}
-
-void PubSubData::setShouldClaim(bool shouldClaim) {
-  shouldClaim = shouldClaim;
-}
-
-void PubSubData::addTriedServer(HostAddress& h) {
-  triedservers.insert(h);
-}
-
-bool PubSubData::hasTriedServer(HostAddress& h) {
-  return triedservers.count(h) > 0;
-}
-
-void PubSubData::clearTriedServers() {
-  triedservers.clear();
-}
-
-OperationCallbackPtr& PubSubData::getCallback() {
-  return callback;
-}
-
-void PubSubData::setCallback(const OperationCallbackPtr& callback) {
-  this->callback = callback;
-}
-
-const std::string& PubSubData::getSubscriberId() const {
-  return subscriberid;
-}
+  long next= ++counter; 
 
-SubscribeRequest::CreateOrAttach PubSubData::getMode() const {
-  return mode;
+  return next;
 }
 
-ClientImplPtr& ClientImpl::Create(const Configuration& conf) {
-  ClientImpl* impl = new ClientImpl(conf);
-    if (LOG.isDebugEnabled()) {
+ClientImplPtr ClientImpl::Create(const Configuration& conf) {
+  ClientImplPtr impl(new ClientImpl(conf));
+  if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Creating Clientimpl " << impl;
   }
+  impl->dispatcher.start();
 
-  return impl->selfptr;
+  return impl;
 }
 
 void ClientImpl::Destroy() {
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "destroying Clientimpl " << this;
   }
-  allchannels_lock.lock();
 
-  shuttingDownFlag = true;
-  for (ChannelMap::iterator iter = allchannels.begin(); iter != allchannels.end(); ++iter ) {
-    (*iter).second->kill();
-  }  
-  allchannels.clear();
-  allchannels_lock.unlock();
+  dispatcher.stop();
+  {
+    boost::lock_guard<boost::shared_mutex> lock(allchannels_lock);
+    
+    shuttingDownFlag = true;
+    for (ChannelMap::iterator iter = allchannels.begin(); iter != allchannels.end(); ++iter ) {
+      (*iter)->kill();
+    }  
+    allchannels.clear();
+  }
+
   /* destruction of the maps will clean up any items they hold */
   
   if (subscriber != NULL) {
@@ -331,12 +210,10 @@ void ClientImpl::Destroy() {
     delete publisher;
     publisher = NULL;
   }
-
-  selfptr = ClientImplPtr(); // clear the self pointer
 }
 
 ClientImpl::ClientImpl(const Configuration& conf) 
-  : selfptr(this), conf(conf), subscriber(NULL), publisher(NULL), counterobj(), shuttingDownFlag(false)
+  : conf(conf), publisher(NULL), subscriber(NULL), counterobj(), shuttingDownFlag(false)
 {
 }
 
@@ -350,22 +227,20 @@ Publisher& ClientImpl::getPublisher() {
     
 SubscriberImpl& ClientImpl::getSubscriberImpl() {
   if (subscriber == NULL) {
-    subscribercreate_lock.lock();
+    boost::lock_guard<boost::mutex> lock(subscribercreate_lock);
     if (subscriber == NULL) {
-      subscriber = new SubscriberImpl(selfptr);
+      subscriber = new SubscriberImpl(shared_from_this());
     }
-    subscribercreate_lock.unlock();
   }
   return *subscriber;
 }
 
 PublisherImpl& ClientImpl::getPublisherImpl() {
   if (publisher == NULL) { 
-    publishercreate_lock.lock();
+    boost::lock_guard<boost::mutex> lock(publishercreate_lock);
     if (publisher == NULL) {
-      publisher = new PublisherImpl(selfptr);
+      publisher = new PublisherImpl(shared_from_this());
     }
-    publishercreate_lock.unlock();
   }
   return *publisher;
 }
@@ -374,18 +249,21 @@ ClientTxnCounter& ClientImpl::counter() {
   return counterobj;
 }
 
-void ClientImpl::redirectRequest(DuplexChannel* channel, PubSubDataPtr& data, const PubSubResponse& response) {
+void ClientImpl::redirectRequest(const DuplexChannelPtr& channel, PubSubDataPtr& data, const PubSubResponsePtr& response) {
   HostAddress oldhost = channel->getHostAddress();
   data->addTriedServer(oldhost);
   
-  HostAddress h = HostAddress::fromString(response.statusmsg());
+  HostAddress h = HostAddress::fromString(response->statusmsg());
   if (data->hasTriedServer(h)) {
-    LOG.errorStream() << "We've been told to try request [" << data->getTxnId() << "] with [" << h.getAddressString()<< "] by " << channel->getHostAddress().getAddressString() << " but we've already tried that. Failing operation";
+    LOG.errorStream() << "We've been told to try request [" << data->getTxnId() << "] with [" 
+		      << h.getAddressString()<< "] by " << oldhost.getAddressString() 
+		      << " but we've already tried that. Failing operation";
     data->getCallback()->operationFailed(InvalidRedirectException());
     return;
   }
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "We've been told  [" << data->getTopic() << "] is on [" << h.getAddressString() << "] by [" << oldhost.getAddressString() << "]. Redirecting request " << data->getTxnId();
+    LOG.debugStream() << "We've been told  [" << data->getTopic() << "] is on [" << h.getAddressString() 
+		      << "] by [" << oldhost.getAddressString() << "]. Redirecting request " << data->getTxnId();
   }
   data->setShouldClaim(true);
 
@@ -393,21 +271,17 @@ void ClientImpl::redirectRequest(DuplexChannel* channel, PubSubDataPtr& data, co
   DuplexChannelPtr newchannel;
   try {
     if (data->getType() == SUBSCRIBE) {
-      SubscriberClientChannelHandlerPtr handler(new SubscriberClientChannelHandler(selfptr, this->getSubscriberImpl(), data));
-      ChannelHandlerPtr basehandler = handler;
-      
-      newchannel = createChannelForTopic(data->getTopic(), basehandler);
+      SubscriberClientChannelHandlerPtr handler(new SubscriberClientChannelHandler(shared_from_this(), 
+										   this->getSubscriberImpl(), data));
+      newchannel = createChannel(data->getTopic(), handler);
       handler->setChannel(newchannel);
-      
       getSubscriberImpl().doSubscribe(newchannel, data, handler);
+    } else if (data->getType() == PUBLISH) {
+      newchannel = getChannel(data->getTopic());
+      getPublisherImpl().doPublish(newchannel, data);
     } else {
-      newchannel = getChannelForTopic(data->getTopic());
-      
-      if (data->getType() == PUBLISH) {
-	getPublisherImpl().doPublish(newchannel, data);
-      } else {
-	getSubscriberImpl().doUnsubscribe(newchannel, data);
-      }
+      newchannel = getChannel(data->getTopic());
+      getSubscriberImpl().doUnsubscribe(newchannel, data);
     }
   } catch (ShuttingDownException& e) {
     return; // no point in redirecting if we're shutting down
@@ -420,52 +294,57 @@ ClientImpl::~ClientImpl() {
   }
 }
 
-DuplexChannelPtr ClientImpl::createChannelForTopic(const std::string& topic, ChannelHandlerPtr& handler) {
+DuplexChannelPtr ClientImpl::createChannel(const std::string& topic, const ChannelHandlerPtr& handler) {
   // get the host address
   // create a channel to the host
   HostAddress addr = topic2host[topic];
   if (addr.isNullHost()) {
-    addr = HostAddress::fromString(conf.getDefaultServer());
+    addr = HostAddress::fromString(conf.get(Configuration::DEFAULT_SERVER, DEFAULT_SERVER_DEFAULT_VAL));
+    setHostForTopic(topic, addr);
   }
 
-  DuplexChannelPtr channel(new DuplexChannel(addr, conf, handler));
-  channel->connect();
+  DuplexChannelPtr channel(new DuplexChannel(dispatcher, addr, conf, handler));
 
-  allchannels_lock.lock();
+  boost::lock_guard<boost::shared_mutex> lock(allchannels_lock);
   if (shuttingDownFlag) {
     channel->kill();
-    allchannels_lock.unlock();
     throw ShuttingDownException();
   }
-  allchannels[channel.get()] = channel;
+  channel->connect();
+
+  allchannels.insert(channel);
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "(create) All channels size: " << allchannels.size();
   }
-  allchannels_lock.unlock();
 
   return channel;
 }
 
-DuplexChannelPtr ClientImpl::getChannelForTopic(const std::string& topic) {
+DuplexChannelPtr ClientImpl::getChannel(const std::string& topic) {
   HostAddress addr = topic2host[topic];
+  if (addr.isNullHost()) {
+    addr = HostAddress::fromString(conf.get(Configuration::DEFAULT_SERVER, DEFAULT_SERVER_DEFAULT_VAL));
+    setHostForTopic(topic, addr);
+  }  
   DuplexChannelPtr channel = host2channel[addr];
 
-  if (channel.get() == 0 || addr.isNullHost()) {
-    ChannelHandlerPtr handler(new HedwigClientChannelHandler(selfptr));
-    channel = createChannelForTopic(topic, handler);
-    host2channel_lock.lock();
+  if (channel.get() == 0) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debugStream() << " No channel for topic, creating new channel.get() " << channel.get() << " addr " << addr.getAddressString();
+    }
+    ChannelHandlerPtr handler(new HedwigClientChannelHandler(shared_from_this()));
+    channel = createChannel(topic, handler);
+
+    boost::lock_guard<boost::shared_mutex> lock(host2channel_lock);
     host2channel[addr] = channel;
-    host2channel_lock.unlock();
-    return channel;
-  }
+  } 
 
   return channel;
 }
 
 void ClientImpl::setHostForTopic(const std::string& topic, const HostAddress& host) {
-  topic2host_lock.lock();
+  boost::lock_guard<boost::shared_mutex> lock(topic2host_lock);
   topic2host[topic] = host;
-  topic2host_lock.unlock();
 }
 
 bool ClientImpl::shuttingDown() const {
@@ -478,15 +357,15 @@ bool ClientImpl::shuttingDown() const {
    This does not delete the channel. Some publishers or subscribers will still hold it and will be errored
    when they try to do anything with it. 
 */
-void ClientImpl::channelDied(DuplexChannel* channel) {
+void ClientImpl::channelDied(const DuplexChannelPtr& channel) {
   if (shuttingDownFlag) {
     return;
   }
 
-  host2topics_lock.lock();
-  host2channel_lock.lock();
-  topic2host_lock.lock();
-  allchannels_lock.lock();
+  boost::lock_guard<boost::shared_mutex> h2tlock(host2topics_lock);
+  boost::lock_guard<boost::shared_mutex> h2clock(host2channel_lock);
+  boost::lock_guard<boost::shared_mutex> t2hlock(topic2host_lock);
+  boost::lock_guard<boost::shared_mutex> aclock(allchannels_lock);
   // get host
   HostAddress addr = channel->getHostAddress();
   
@@ -497,9 +376,12 @@ void ClientImpl::channelDied(DuplexChannel* channel) {
   host2channel.erase(addr);
 
   allchannels.erase(channel); // channel should be deleted here
+}
+
+const Configuration& ClientImpl::getConfiguration() {
+  return conf;
+}
 
-  allchannels_lock.unlock();
-  host2topics_lock.unlock();
-  host2channel_lock.unlock();
-  topic2host_lock.unlock();
+boost::asio::io_service& ClientImpl::getService() {
+  return dispatcher.getService();
 }

+ 51 - 32
src/contrib/hedwig/client/src/main/cpp/lib/clientimpl.h

@@ -22,22 +22,30 @@
 #include <hedwig/client.h>
 #include <hedwig/protocol.h>
 
+#include <boost/asio.hpp>
+#include <boost/shared_ptr.hpp>
+#include <boost/enable_shared_from_this.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/condition_variable.hpp>
+
 #include <tr1/unordered_map>
 #include <list>
+
 #include "util.h"
-#include <pthread.h>
 #include "channel.h"
 #include "data.h"
+#include "eventdispatcher.h"
 
 namespace Hedwig {
-  class SyncOperationCallback : public OperationCallback, public WaitConditionBase {
+  const int DEFAULT_SYNC_REQUEST_TIMEOUT = 5000;
+
+  class SyncOperationCallback : public OperationCallback {
   public:
-    SyncOperationCallback() : response(PENDING) {}
+  SyncOperationCallback(int timeout) : response(PENDING), timeout(timeout) {}
     virtual void operationComplete();
     virtual void operationFailed(const std::exception& exception);
     
-    virtual bool isTrue();
-
+    void wait();
     void throwExceptionIfNeeded();
     
   private:
@@ -48,21 +56,26 @@ namespace Hedwig {
       SERVICEDOWN,
       NOT_SUBSCRIBED,
       ALREADY_SUBSCRIBED,
+      TIMEOUT,
       UNKNOWN
     } response;
+    
+    boost::condition_variable cond;
+    boost::mutex mut;
+    int timeout;
   };
 
   class HedwigClientChannelHandler : public ChannelHandler {
   public:
-    HedwigClientChannelHandler(ClientImplPtr& client);
+    HedwigClientChannelHandler(const ClientImplPtr& client);
     
-    virtual void messageReceived(DuplexChannel* channel, const PubSubResponse& m);
-    virtual void channelConnected(DuplexChannel* channel);
-    virtual void channelDisconnected(DuplexChannel* channel, const std::exception& e);
-    virtual void exceptionOccurred(DuplexChannel* channel, const std::exception& e);
+    virtual void messageReceived(const DuplexChannelPtr& channel, const PubSubResponsePtr& m);
+    virtual void channelConnected(const DuplexChannelPtr& channel);
+    virtual void channelDisconnected(const DuplexChannelPtr& channel, const std::exception& e);
+    virtual void exceptionOccurred(const DuplexChannelPtr& channel, const std::exception& e);
     
   protected:
-    ClientImplPtr client;
+    const ClientImplPtr client;
   };
   
   class PublisherImpl;
@@ -71,9 +84,9 @@ namespace Hedwig {
   /**
      Implementation of the hedwig client. This class takes care of globals such as the topic->host map and the transaction id counter.
   */
-  class ClientImpl {
+  class ClientImpl : public boost::enable_shared_from_this<ClientImpl> {
   public:
-    static ClientImplPtr& Create(const Configuration& conf);
+    static ClientImplPtr Create(const Configuration& conf);
     void Destroy();
 
     Subscriber& getSubscriber();
@@ -81,49 +94,55 @@ namespace Hedwig {
 
     ClientTxnCounter& counter();
 
-    void redirectRequest(DuplexChannel* channel, PubSubDataPtr& data, const PubSubResponse& response);
+    void redirectRequest(const DuplexChannelPtr& channel, PubSubDataPtr& data, const PubSubResponsePtr& response);
 
     const HostAddress& getHostForTopic(const std::string& topic);
 
-    DuplexChannelPtr createChannelForTopic(const std::string& topic, ChannelHandlerPtr& handler);
-    DuplexChannelPtr getChannelForTopic(const std::string& topic);
-    
+    //DuplexChannelPtr getChannelForTopic(const std::string& topic, OperationCallback& callback);
+    //DuplexChannelPtr createChannelForTopic(const std::string& topic, ChannelHandlerPtr& handler, OperationCallback& callback);
+    DuplexChannelPtr createChannel(const std::string& topic, const ChannelHandlerPtr& handler);    
+    DuplexChannelPtr getChannel(const std::string& topic);
+
     void setHostForTopic(const std::string& topic, const HostAddress& host);
 
-    void setChannelForHost(const HostAddress& address, DuplexChannel* channel);
-    void channelDied(DuplexChannel* channel);
+    void setChannelForHost(const HostAddress& address, const DuplexChannelPtr& channel);
+    void channelDied(const DuplexChannelPtr& channel);
     bool shuttingDown() const;
     
     SubscriberImpl& getSubscriberImpl();
     PublisherImpl& getPublisherImpl();
 
+    const Configuration& getConfiguration();
+    boost::asio::io_service& getService();
+
     ~ClientImpl();
   private:
     ClientImpl(const Configuration& conf);
 
-    ClientImplPtr selfptr;
-
     const Configuration& conf;
+
+    boost::mutex publishercreate_lock;
     PublisherImpl* publisher;
+
+    boost::mutex subscribercreate_lock;
     SubscriberImpl* subscriber;
-    ClientTxnCounter counterobj;
 
+    ClientTxnCounter counterobj;
 
-    typedef std::tr1::unordered_multimap<HostAddress, std::string> Host2TopicsMap;
+    EventDispatcher dispatcher;
+    
+    typedef std::tr1::unordered_multimap<HostAddress, std::string, HostAddressHash > Host2TopicsMap;
     Host2TopicsMap host2topics;
-    Mutex host2topics_lock;
+    boost::shared_mutex host2topics_lock;
 
-    std::tr1::unordered_map<HostAddress, DuplexChannelPtr> host2channel;
-    Mutex host2channel_lock;
+    std::tr1::unordered_map<HostAddress, DuplexChannelPtr, HostAddressHash > host2channel;
+    boost::shared_mutex host2channel_lock;
     std::tr1::unordered_map<std::string, HostAddress> topic2host;
-    Mutex topic2host_lock;
-
-    Mutex publishercreate_lock;
-    Mutex subscribercreate_lock;
+    boost::shared_mutex topic2host_lock;
 
-    typedef std::tr1::unordered_map<DuplexChannel*, DuplexChannelPtr> ChannelMap;
+    typedef std::tr1::unordered_set<DuplexChannelPtr, DuplexChannelPtrHash > ChannelMap;
     ChannelMap allchannels;
-    Mutex allchannels_lock;
+    boost::shared_mutex allchannels_lock;
 
     bool shuttingDownFlag;
   };

+ 173 - 0
src/contrib/hedwig/client/src/main/cpp/lib/data.cpp

@@ -0,0 +1,173 @@
+/**
+ * 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 <hedwig/protocol.h>
+#include "data.h"
+
+#include <log4cpp/Category.hh>
+
+static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__);
+
+using namespace Hedwig;
+
+PubSubDataPtr PubSubData::forPublishRequest(long txnid, const std::string& topic, const std::string& body, const OperationCallbackPtr& callback) {
+  PubSubDataPtr ptr(new PubSubData());
+  ptr->type = PUBLISH;
+  ptr->txnid = txnid;
+  ptr->topic = topic;
+  ptr->body = body;
+  ptr->callback = callback;
+  return ptr;
+}
+
+PubSubDataPtr PubSubData::forSubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const OperationCallbackPtr& callback, SubscribeRequest::CreateOrAttach mode) {
+  PubSubDataPtr ptr(new PubSubData());
+  ptr->type = SUBSCRIBE;
+  ptr->txnid = txnid;
+  ptr->subscriberid = subscriberid;
+  ptr->topic = topic;
+  ptr->callback = callback;
+  ptr->mode = mode;
+  return ptr;  
+}
+
+PubSubDataPtr PubSubData::forUnsubscribeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const OperationCallbackPtr& callback) {
+  PubSubDataPtr ptr(new PubSubData());
+  ptr->type = UNSUBSCRIBE;
+  ptr->txnid = txnid;
+  ptr->subscriberid = subscriberid;
+  ptr->topic = topic;
+  ptr->callback = callback;
+  return ptr;  
+}
+
+PubSubDataPtr PubSubData::forConsumeRequest(long txnid, const std::string& subscriberid, const std::string& topic, const MessageSeqId msgid) {
+  PubSubDataPtr ptr(new PubSubData());
+  ptr->type = CONSUME;
+  ptr->txnid = txnid;
+  ptr->subscriberid = subscriberid;
+  ptr->topic = topic;
+  ptr->msgid = msgid;
+  return ptr;  
+}
+
+PubSubData::PubSubData() : shouldClaim(false) {  
+}
+
+PubSubData::~PubSubData() {
+}
+
+OperationType PubSubData::getType() const {
+  return type;
+}
+
+long PubSubData::getTxnId() const {
+  return txnid;
+}
+
+const std::string& PubSubData::getTopic() const {
+  return topic;
+}
+
+const std::string& PubSubData::getBody() const {
+  return body;
+}
+
+const MessageSeqId PubSubData::getMessageSeqId() const {
+  return msgid;
+}
+
+const PubSubRequestPtr PubSubData::getRequest() {
+  PubSubRequestPtr request(new Hedwig::PubSubRequest());
+  request->set_protocolversion(Hedwig::VERSION_ONE);
+  request->set_type(type);
+  request->set_txnid(txnid);
+  if (shouldClaim) {
+    request->set_shouldclaim(shouldClaim);
+  }
+  request->set_topic(topic);
+    
+  if (type == PUBLISH) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debugStream() << "Creating publish request";
+    }
+    Hedwig::PublishRequest* pubreq = request->mutable_publishrequest();
+    Hedwig::Message* msg = pubreq->mutable_msg();
+    msg->set_body(body);
+  } else if (type == SUBSCRIBE) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debugStream() << "Creating subscribe request";
+    }
+
+    Hedwig::SubscribeRequest* subreq = request->mutable_subscriberequest();
+    subreq->set_subscriberid(subscriberid);
+    subreq->set_createorattach(mode);
+  } else if (type == CONSUME) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debugStream() << "Creating consume request";
+    }
+
+    Hedwig::ConsumeRequest* conreq = request->mutable_consumerequest();
+    conreq->set_subscriberid(subscriberid);
+    conreq->mutable_msgid()->CopyFrom(msgid);
+  } else if (type == UNSUBSCRIBE) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debugStream() << "Creating unsubscribe request";
+    }
+    
+    Hedwig::UnsubscribeRequest* unsubreq = request->mutable_unsubscriberequest();
+    unsubreq->set_subscriberid(subscriberid);    
+  } else {
+    LOG.errorStream() << "Tried to create a request message for the wrong type [" << type << "]";
+    throw UnknownRequestException();
+  }
+
+  return request;
+}
+
+void PubSubData::setShouldClaim(bool shouldClaim) {
+  this->shouldClaim = shouldClaim;
+}
+
+void PubSubData::addTriedServer(HostAddress& h) {
+  triedservers.insert(h);
+}
+
+bool PubSubData::hasTriedServer(HostAddress& h) {
+  return triedservers.count(h) > 0;
+}
+
+void PubSubData::clearTriedServers() {
+  triedservers.clear();
+}
+
+OperationCallbackPtr& PubSubData::getCallback() {
+  return callback;
+}
+
+void PubSubData::setCallback(const OperationCallbackPtr& callback) {
+  this->callback = callback;
+}
+
+const std::string& PubSubData::getSubscriberId() const {
+  return subscriberid;
+}
+
+SubscribeRequest::CreateOrAttach PubSubData::getMode() const {
+  return mode;
+}

+ 10 - 6
src/contrib/hedwig/client/src/main/cpp/lib/data.h

@@ -25,6 +25,8 @@
 #include <pthread.h>
 #include <tr1/unordered_set>
 #include "util.h"
+#include <boost/shared_ptr.hpp>
+#include <boost/thread/mutex.hpp>
 
 namespace Hedwig {
   /**
@@ -38,11 +40,13 @@ namespace Hedwig {
     
   private:
     long counter;
-    Mutex mutex;
+    boost::mutex mutex;
   };
 
   class PubSubData;
-  typedef std::tr1::shared_ptr<PubSubData> PubSubDataPtr;
+  typedef boost::shared_ptr<PubSubData> PubSubDataPtr;
+  typedef boost::shared_ptr<PubSubRequest> PubSubRequestPtr;
+  typedef boost::shared_ptr<PubSubResponse> PubSubResponsePtr;
 
   /**
      Data structure to hold information about requests and build request messages.
@@ -63,10 +67,11 @@ namespace Hedwig {
     const std::string& getSubscriberId() const;
     const std::string& getTopic() const;
     const std::string& getBody() const;
+    const MessageSeqId getMessageSeqId() const;
 
     void setShouldClaim(bool shouldClaim);
 
-    const PubSubRequest& getRequest();
+    const PubSubRequestPtr getRequest();
     void setCallback(const OperationCallbackPtr& callback);
     OperationCallbackPtr& getCallback();
     SubscribeRequest::CreateOrAttach getMode() const;
@@ -75,8 +80,8 @@ namespace Hedwig {
     bool hasTriedServer(HostAddress& h);
     void clearTriedServers();
   private:
+
     PubSubData();
-    PubSubRequest* request;
     
     OperationType type;
     long txnid;
@@ -87,9 +92,8 @@ namespace Hedwig {
     OperationCallbackPtr callback;
     SubscribeRequest::CreateOrAttach mode;
     MessageSeqId msgid;
-    std::tr1::unordered_set<HostAddress> triedservers;
+    std::tr1::unordered_set<HostAddress, HostAddressHash > triedservers;
   };
   
-
 };
 #endif

+ 76 - 0
src/contrib/hedwig/client/src/main/cpp/lib/eventdispatcher.cpp

@@ -0,0 +1,76 @@
+/**
+ * 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 "eventdispatcher.h"
+
+#include <log4cpp/Category.hh>
+
+static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__);
+
+using namespace Hedwig;
+
+EventDispatcher::EventDispatcher() : service(), dummy_work(NULL), t(NULL) {
+}
+
+void EventDispatcher::run_forever() {
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Starting event dispatcher";
+  }
+
+  while (true) {
+    try {
+      service.run();
+      break;
+    } catch (std::exception &e) {
+      LOG.errorStream() << "Exception in dispatch handler. " << e.what();
+    }
+  }
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Event dispatcher done";
+  }
+}
+
+void EventDispatcher::start() {
+  if (t) {
+    return;
+  }
+  dummy_work = new boost::asio::io_service::work(service);
+  t = new boost::thread(boost::bind(&EventDispatcher::run_forever, this));
+}
+
+void EventDispatcher::stop() {
+  if (!t) {
+    return;
+  }
+  delete dummy_work;
+  dummy_work = NULL;
+  
+  service.stop();
+  
+  t->join();
+  delete t;
+  t = NULL;
+}
+
+EventDispatcher::~EventDispatcher() {
+  delete dummy_work;
+}
+
+boost::asio::io_service& EventDispatcher::getService() {
+  return service;
+}

+ 44 - 0
src/contrib/hedwig/client/src/main/cpp/lib/eventdispatcher.h

@@ -0,0 +1,44 @@
+/**
+ * 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.
+ */
+#ifndef EVENTDISPATCHER_H
+#define EVENTDISPATCHER_H
+
+#include <boost/asio.hpp>
+#include <boost/thread.hpp>
+
+namespace Hedwig {
+  class EventDispatcher {
+  public:  
+    EventDispatcher();
+    ~EventDispatcher();
+    
+    void start();
+    void stop();
+    
+    boost::asio::io_service& getService();
+    
+  private:
+    void run_forever();
+
+    boost::asio::io_service service;
+    boost::asio::io_service::work* dummy_work;
+    boost::thread* t;
+  };
+}
+
+#endif

+ 9 - 11
src/contrib/hedwig/client/src/main/cpp/lib/publisherimpl.cpp

@@ -25,7 +25,7 @@ static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__
 
 using namespace Hedwig;
 
-PublishWriteCallback::PublishWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
+PublishWriteCallback::PublishWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
 
 void PublishWriteCallback::operationComplete() {
   if (LOG.isDebugEnabled()) {
@@ -36,16 +36,16 @@ void PublishWriteCallback::operationComplete() {
 void PublishWriteCallback::operationFailed(const std::exception& exception) {
   LOG.errorStream() << "Error writing to publisher " << exception.what();
   
-  //remove txn from channel pending list
-  #warning "Actually do something here"
+  data->getCallback()->operationFailed(exception);
 }
 
-PublisherImpl::PublisherImpl(ClientImplPtr& client) 
+PublisherImpl::PublisherImpl(const ClientImplPtr& client) 
   : client(client) {
 }
 
 void PublisherImpl::publish(const std::string& topic, const std::string& message) {
-  SyncOperationCallback* cb = new SyncOperationCallback();
+  SyncOperationCallback* cb = new SyncOperationCallback(client->getConfiguration().getInt(Configuration::SYNC_REQUEST_TIMEOUT, 
+											  DEFAULT_SYNC_REQUEST_TIMEOUT));
   OperationCallbackPtr callback(cb);
   asyncPublish(topic, message, callback);
   cb->wait();
@@ -54,11 +54,10 @@ void PublisherImpl::publish(const std::string& topic, const std::string& message
 }
 
 void PublisherImpl::asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback) {
-  DuplexChannelPtr channel = client->getChannelForTopic(topic);
-
   // use release after callback to release the channel after the callback is called
   PubSubDataPtr data = PubSubData::forPublishRequest(client->counter().next(), topic, message, callback);
   
+  DuplexChannelPtr channel = client->getChannel(topic);
   doPublish(channel, data);
 }
 
@@ -66,12 +65,11 @@ void PublisherImpl::doPublish(const DuplexChannelPtr& channel, const PubSubDataP
   channel->storeTransaction(data);
   
   OperationCallbackPtr writecb(new PublishWriteCallback(client, data));
-  LOG.debugStream() << "dopublish";
   channel->writeRequest(data->getRequest(), writecb);
 }
 
-void PublisherImpl::messageHandler(const PubSubResponse& m, const PubSubDataPtr& txn) {
-  switch (m.statuscode()) {
+void PublisherImpl::messageHandler(const PubSubResponsePtr& m, const PubSubDataPtr& txn) {
+  switch (m->statuscode()) {
   case SUCCESS:
     txn->getCallback()->operationComplete();
     break;
@@ -80,7 +78,7 @@ void PublisherImpl::messageHandler(const PubSubResponse& m, const PubSubDataPtr&
     txn->getCallback()->operationFailed(ServiceDownException());
     break;
   default:
-    LOG.errorStream() << "Unexpected response " << m.statuscode() << " for " << txn->getTxnId();
+    LOG.errorStream() << "Unexpected response " << m->statuscode() << " for " << txn->getTxnId();
     txn->getCallback()->operationFailed(UnexpectedResponseException());
     break;
   }

+ 3 - 3
src/contrib/hedwig/client/src/main/cpp/lib/publisherimpl.h

@@ -25,7 +25,7 @@
 namespace Hedwig {
   class PublishWriteCallback : public OperationCallback {
   public:
-    PublishWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data);
+    PublishWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data);
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
@@ -36,12 +36,12 @@ namespace Hedwig {
 
   class PublisherImpl : public Publisher {
   public:
-    PublisherImpl(ClientImplPtr& client);
+    PublisherImpl(const ClientImplPtr& client);
 
     void publish(const std::string& topic, const std::string& message);
     void asyncPublish(const std::string& topic, const std::string& message, const OperationCallbackPtr& callback);
     
-    void messageHandler(const PubSubResponse& m, const PubSubDataPtr& txn);
+    void messageHandler(const PubSubResponsePtr& m, const PubSubDataPtr& txn);
 
     void doPublish(const DuplexChannelPtr& channel, const PubSubDataPtr& data);
 

+ 150 - 89
src/contrib/hedwig/client/src/main/cpp/lib/subscriberimpl.cpp

@@ -20,13 +20,20 @@
 #include "util.h"
 #include "channel.h"
 
+#include <boost/asio.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+
 #include <log4cpp/Category.hh>
 
 static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwig."__FILE__);
-const int SUBSCRIBER_RECONNECT_TIME = 3000; // 3 seconds
+
 using namespace Hedwig;
+const int DEFAULT_MESSAGE_CONSUME_RETRY_WAIT_TIME = 5000;
+const int DEFAULT_SUBSCRIBER_CONSUME_RETRY_WAIT_TIME = 5000;
+const int DEFAULT_MAX_MESSAGE_QUEUE_SIZE = 10;
+const int DEFAULT_RECONNECT_SUBSCRIBE_RETRY_WAIT_TIME = 5000;
 
-SubscriberWriteCallback::SubscriberWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
+SubscriberWriteCallback::SubscriberWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
 
 void SubscriberWriteCallback::operationComplete() {
   if (LOG.isDebugEnabled()) {
@@ -35,29 +42,43 @@ void SubscriberWriteCallback::operationComplete() {
 }
 
 void SubscriberWriteCallback::operationFailed(const std::exception& exception) {
-  LOG.errorStream() << "Error writing to publisher " << exception.what();
+  LOG.errorStream() << "Error writing to subscriber " << exception.what();
   
   //remove txn from channel pending list
-  #warning "Actually do something here"
+  data->getCallback()->operationFailed(exception);
+  client->getSubscriberImpl().closeSubscription(data->getTopic(), data->getSubscriberId());
 }
 
-UnsubscribeWriteCallback::UnsubscribeWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
+UnsubscribeWriteCallback::UnsubscribeWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data) : client(client), data(data) {}
 
 void UnsubscribeWriteCallback::operationComplete() {
-  
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "Successfully wrote unsubscribe transaction: " << data->getTxnId();
+  }  
 }
 
 void UnsubscribeWriteCallback::operationFailed(const std::exception& exception) {
-  #warning "Actually do something here"
+  data->getCallback()->operationFailed(exception);
 }
   
-ConsumeWriteCallback::ConsumeWriteCallback(const PubSubDataPtr& data) 
-  : data(data) {
+ConsumeWriteCallback::ConsumeWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data) 
+  : client(client), data(data) {
 }
 
 ConsumeWriteCallback::~ConsumeWriteCallback() {
 }
 
+/* static */ void ConsumeWriteCallback::timerComplete(const ClientImplPtr& client, const PubSubDataPtr& data,
+						      const boost::system::error_code& error) {
+  if (error) {
+    // shutting down
+    return;
+  }
+
+  client->getSubscriberImpl().consume(data->getTopic(), data->getSubscriberId(), data->getMessageSeqId());
+}
+
+
 void ConsumeWriteCallback::operationComplete() {
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Successfully wrote consume transaction: " << data->getTxnId();
@@ -65,24 +86,54 @@ void ConsumeWriteCallback::operationComplete() {
 }
 
 void ConsumeWriteCallback::operationFailed(const std::exception& exception) {
-  LOG.errorStream() << "Error writing consume transaction: " << data->getTxnId() << " error: " << exception.what();
+  int retrywait = client->getConfiguration().getInt(Configuration::MESSAGE_CONSUME_RETRY_WAIT_TIME, 
+						    DEFAULT_MESSAGE_CONSUME_RETRY_WAIT_TIME);
+  LOG.errorStream() << "Error writing consume transaction: " << data->getTxnId() << " error: " << exception.what() 
+		    << " retrying in " << retrywait << " Microseconds";
+
+  boost::asio::deadline_timer t(client->getService(), boost::posix_time::milliseconds(retrywait));
+
+  t.async_wait(boost::bind(&ConsumeWriteCallback::timerComplete, client, data, boost::asio::placeholders::error));  
 }
 
-SubscriberConsumeCallback::SubscriberConsumeCallback(ClientImplPtr& client, const std::string& topic, const std::string& subscriberid, const MessageSeqId& msgid) 
-  : client(client), topic(topic), subscriberid(subscriberid), msgid(msgid)
+SubscriberConsumeCallback::SubscriberConsumeCallback(const ClientImplPtr& client, 
+						     const SubscriberClientChannelHandlerPtr& handler, 
+						     const PubSubDataPtr& data, const PubSubResponsePtr& m) 
+  : client(client), handler(handler), data(data), m(m)
 {
 }
 
 void SubscriberConsumeCallback::operationComplete() {
-  LOG.errorStream() << "ConsumeCallback::operationComplete";
-  client->getSubscriber().consume(topic, subscriberid, msgid);
+  if (LOG.isDebugEnabled()) {
+    LOG.debugStream() << "ConsumeCallback::operationComplete " << data->getTopic() << " - " << data->getSubscriberId();
+  };
+  client->getSubscriber().consume(data->getTopic(), data->getSubscriberId(), m->message().msgid());
+}
+
+/* static */ void SubscriberConsumeCallback::timerComplete(const SubscriberClientChannelHandlerPtr handler, 
+							   const PubSubResponsePtr m, 
+							   const boost::system::error_code& error) {
+  if (error) {
+    return;
+  }
+  handler->messageReceived(handler->getChannel(), m);
 }
 
 void SubscriberConsumeCallback::operationFailed(const std::exception& exception) {
-  LOG.errorStream() << "ConsumeCallback::operationFailed";
+  LOG.errorStream() << "ConsumeCallback::operationFailed  " << data->getTopic() << " - " << data->getSubscriberId();
+  
+  int retrywait = client->getConfiguration().getInt(Configuration::SUBSCRIBER_CONSUME_RETRY_WAIT_TIME,
+						    DEFAULT_SUBSCRIBER_CONSUME_RETRY_WAIT_TIME);
+
+  LOG.errorStream() << "Error passing message to client transaction: " << data->getTxnId() << " error: " << exception.what() 
+		    << " retrying in " << retrywait << " Microseconds";
+
+  boost::asio::deadline_timer t(client->getService(), boost::posix_time::milliseconds(retrywait));
+
+  t.async_wait(boost::bind(&SubscriberConsumeCallback::timerComplete, handler, m, boost::asio::placeholders::error));  
 }
 
-SubscriberReconnectCallback::SubscriberReconnectCallback(ClientImplPtr& client, const PubSubDataPtr& origData) 
+SubscriberReconnectCallback::SubscriberReconnectCallback(const ClientImplPtr& client, const PubSubDataPtr& origData) 
   : client(client), origData(origData) {
 }
 
@@ -90,11 +141,13 @@ void SubscriberReconnectCallback::operationComplete() {
 }
 
 void SubscriberReconnectCallback::operationFailed(const std::exception& exception) {
-  
+  LOG.errorStream() << "Error writing to new subscriber. Channel should pick this up disconnect the channel and try to connect again " << exception.what();
+
+
 }
 
-SubscriberClientChannelHandler::SubscriberClientChannelHandler(ClientImplPtr& client, SubscriberImpl& subscriber, const PubSubDataPtr& data)
-  : HedwigClientChannelHandler(client), subscriber(subscriber), origData(data), closed(false)  {
+SubscriberClientChannelHandler::SubscriberClientChannelHandler(const ClientImplPtr& client, SubscriberImpl& subscriber, const PubSubDataPtr& data)
+  : HedwigClientChannelHandler(client), subscriber(subscriber), origData(data), closed(false), should_wait(true)  {
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Creating SubscriberClientChannelHandler " << this;
   }
@@ -106,18 +159,21 @@ SubscriberClientChannelHandler::~SubscriberClientChannelHandler() {
   }
 }
 
-void SubscriberClientChannelHandler::messageReceived(DuplexChannel* channel, const PubSubResponse& m) {
-  if (m.has_message()) {
+void SubscriberClientChannelHandler::messageReceived(const DuplexChannelPtr& channel, const PubSubResponsePtr& m) {
+  if (m->has_message()) {
     if (LOG.isDebugEnabled()) {
       LOG.debugStream() << "Message received (topic:" << origData->getTopic() << ", subscriberId:" << origData->getSubscriberId() << ")";
     }
 
     if (this->handler.get()) {
-      OperationCallbackPtr callback(new SubscriberConsumeCallback(client, origData->getTopic(), origData->getSubscriberId(), m.message().msgid()));
-      this->handler->consume(origData->getTopic(), origData->getSubscriberId(), m.message(), callback);
+      OperationCallbackPtr callback(new SubscriberConsumeCallback(client, shared_from_this(), origData, m));
+      this->handler->consume(origData->getTopic(), origData->getSubscriberId(), m->message(), callback);
     } else {
-      LOG.debugStream() << "putting in queue";
-      queue.push_back(m.message());
+      queue.push_back(m);
+      if (queue.size() >= (std::size_t)client->getConfiguration().getInt(Configuration::MAX_MESSAGE_QUEUE_SIZE,
+									 DEFAULT_MAX_MESSAGE_QUEUE_SIZE)) {
+	channel->stopReceiving();
+      }
     }
   } else {
     HedwigClientChannelHandler::messageReceived(channel, m);
@@ -126,12 +182,23 @@ void SubscriberClientChannelHandler::messageReceived(DuplexChannel* channel, con
 
 void SubscriberClientChannelHandler::close() {
   closed = true;
+
   if (channel) {
     channel->kill();
   }
 }
 
-void SubscriberClientChannelHandler::channelDisconnected(DuplexChannel* channel, const std::exception& e) {
+/*static*/ void SubscriberClientChannelHandler::reconnectTimerComplete(const SubscriberClientChannelHandlerPtr handler,
+								       const DuplexChannelPtr channel, const std::exception e, 
+								       const boost::system::error_code& error) {
+  if (error) {
+    return;
+  }
+  handler->should_wait = false;
+  handler->channelDisconnected(channel, e);
+}
+
+void SubscriberClientChannelHandler::channelDisconnected(const DuplexChannelPtr& channel, const std::exception& e) {
   // has subscription been closed
   if (closed) {
     return;
@@ -142,59 +209,60 @@ void SubscriberClientChannelHandler::channelDisconnected(DuplexChannel* channel,
   if (client->shuttingDown()) {
     return;
   }
-  
+
+  if (should_wait) {
+    int retrywait = client->getConfiguration().getInt(Configuration::RECONNECT_SUBSCRIBE_RETRY_WAIT_TIME,
+						      DEFAULT_RECONNECT_SUBSCRIBE_RETRY_WAIT_TIME);
+    
+    boost::asio::deadline_timer t(client->getService(), boost::posix_time::milliseconds(retrywait));
+    t.async_wait(boost::bind(&SubscriberClientChannelHandler::reconnectTimerComplete, shared_from_this(), 
+			     channel, e, boost::asio::placeholders::error));  
+
+  }
+  should_wait = true;
+
   // setup pubsub data for reconnection attempt
   origData->clearTriedServers();
   OperationCallbackPtr newcallback(new SubscriberReconnectCallback(client, origData));
   origData->setCallback(newcallback);
 
   // Create a new handler for the new channel
-  SubscriberClientChannelHandlerPtr handler(new SubscriberClientChannelHandler(client, subscriber, origData));  
-  ChannelHandlerPtr baseptr = handler;
-  // if there is an error createing the channel, sleep for SUBSCRIBER_RECONNECT_TIME and try again
-  DuplexChannelPtr newchannel;
-  while (true) {
-    try {
-      newchannel = client->createChannelForTopic(origData->getTopic(), baseptr);
-      handler->setChannel(newchannel);
-      break;
-    } catch (ShuttingDownException& e) {
-      LOG.errorStream() << "Shutting down, don't try to reconnect";
-      return; 
-    } catch (ChannelException& e) {
-      LOG.errorStream() << "Couldn't acquire channel, sleeping for " << SUBSCRIBER_RECONNECT_TIME << " before trying again";
-      usleep(SUBSCRIBER_RECONNECT_TIME);
-    }
-  } 
-  handoverDelivery(handler.get());
+  SubscriberClientChannelHandlerPtr newhandler(new SubscriberClientChannelHandler(client, subscriber, origData));  
+  ChannelHandlerPtr baseptr = newhandler;
+  
+  DuplexChannelPtr newchannel = client->createChannel(origData->getTopic(), baseptr);
+  newhandler->setChannel(newchannel);
+  handoverDelivery(newhandler);
   
   // remove record of the failed channel from the subscriber
-  subscriber.closeSubscription(origData->getTopic(), origData->getSubscriberId());
-
+  client->getSubscriberImpl().closeSubscription(origData->getTopic(), origData->getSubscriberId());
+  
   // subscriber
-  subscriber.doSubscribe(newchannel, origData, handler);
+  client->getSubscriberImpl().doSubscribe(newchannel, origData, newhandler);
 }
 
 void SubscriberClientChannelHandler::startDelivery(const MessageHandlerCallbackPtr& handler) {
   this->handler = handler;
   
   while (!queue.empty()) {    
-    LOG.debugStream() << "Taking from queue";
-    Message m = queue.front();
+    PubSubResponsePtr m = queue.front();
     queue.pop_front();
 
-    OperationCallbackPtr callback(new SubscriberConsumeCallback(client, origData->getTopic(), origData->getSubscriberId(), m.msgid()));
+    OperationCallbackPtr callback(new SubscriberConsumeCallback(client, shared_from_this(), origData, m));
 
-    this->handler->consume(origData->getTopic(), origData->getSubscriberId(), m, callback);
+    this->handler->consume(origData->getTopic(), origData->getSubscriberId(), m->message(), callback);
   }
+  channel->startReceiving();
 }
 
 void SubscriberClientChannelHandler::stopDelivery() {
+  channel->stopReceiving();
+
   this->handler = MessageHandlerCallbackPtr();
 }
 
 
-void SubscriberClientChannelHandler::handoverDelivery(SubscriberClientChannelHandler* newHandler) {
+void SubscriberClientChannelHandler::handoverDelivery(const SubscriberClientChannelHandlerPtr& newHandler) {
   LOG.debugStream() << "Messages in queue " << queue.size();
   MessageHandlerCallbackPtr handler = this->handler;
   stopDelivery(); // resets old handler
@@ -209,7 +277,7 @@ DuplexChannelPtr& SubscriberClientChannelHandler::getChannel() {
   return channel;
 }
 
-SubscriberImpl::SubscriberImpl(ClientImplPtr& client) 
+SubscriberImpl::SubscriberImpl(const ClientImplPtr& client) 
   : client(client) 
 {
 }
@@ -221,7 +289,8 @@ SubscriberImpl::~SubscriberImpl()
 
 
 void SubscriberImpl::subscribe(const std::string& topic, const std::string& subscriberId, const SubscribeRequest::CreateOrAttach mode) {
-  SyncOperationCallback* cb = new SyncOperationCallback();
+  SyncOperationCallback* cb = new SyncOperationCallback(client->getConfiguration().getInt(Configuration::SYNC_REQUEST_TIMEOUT, 
+											  DEFAULT_SYNC_REQUEST_TIMEOUT));
   OperationCallbackPtr callback(cb);
   asyncSubscribe(topic, subscriberId, mode, callback);
   cb->wait();
@@ -232,37 +301,35 @@ void SubscriberImpl::subscribe(const std::string& topic, const std::string& subs
 void SubscriberImpl::asyncSubscribe(const std::string& topic, const std::string& subscriberId, const SubscribeRequest::CreateOrAttach mode, const OperationCallbackPtr& callback) {
   PubSubDataPtr data = PubSubData::forSubscribeRequest(client->counter().next(), subscriberId, topic, callback, mode);
 
-  SubscriberClientChannelHandlerPtr handler(new SubscriberClientChannelHandler(client, *this, data));  
+  SubscriberClientChannelHandlerPtr handler(new SubscriberClientChannelHandler(client, *this, data));
   ChannelHandlerPtr baseptr = handler;
-  DuplexChannelPtr channel = client->createChannelForTopic(topic, baseptr);
-  
-  handler->setChannel(channel);
 
+  DuplexChannelPtr channel = client->createChannel(topic, handler);
+  handler->setChannel(channel);
   doSubscribe(channel, data, handler);
 }
 
 void SubscriberImpl::doSubscribe(const DuplexChannelPtr& channel, const PubSubDataPtr& data, const SubscriberClientChannelHandlerPtr& handler) {
-  LOG.debugStream() << "doSubscribe";
   channel->storeTransaction(data);
 
   OperationCallbackPtr writecb(new SubscriberWriteCallback(client, data));
   channel->writeRequest(data->getRequest(), writecb);
 
-  topicsubscriber2handler_lock.lock();
+  boost::lock_guard<boost::shared_mutex> lock(topicsubscriber2handler_lock);
   TopicSubscriber t(data->getTopic(), data->getSubscriberId());
   SubscriberClientChannelHandlerPtr oldhandler = topicsubscriber2handler[t];
   if (oldhandler != NULL) {
-    oldhandler->handoverDelivery(handler.get());
+    oldhandler->handoverDelivery(handler);
   }
   topicsubscriber2handler[t] = handler;
   if (LOG.isDebugEnabled()) {
     LOG.debugStream() << "Set topic subscriber for topic(" << data->getTopic() << ") subscriberId(" << data->getSubscriberId() << ") to " << handler.get() << " topicsubscriber2topic(" << &topicsubscriber2handler << ")";
   }
-  topicsubscriber2handler_lock.unlock();;
 }
 
 void SubscriberImpl::unsubscribe(const std::string& topic, const std::string& subscriberId) {
-  SyncOperationCallback* cb = new SyncOperationCallback();
+  SyncOperationCallback* cb = new SyncOperationCallback(client->getConfiguration().getInt(Configuration::SYNC_REQUEST_TIMEOUT, 
+											  DEFAULT_SYNC_REQUEST_TIMEOUT));
   OperationCallbackPtr callback(cb);
   asyncUnsubscribe(topic, subscriberId, callback);
   cb->wait();
@@ -275,14 +342,8 @@ void SubscriberImpl::asyncUnsubscribe(const std::string& topic, const std::strin
 
   PubSubDataPtr data = PubSubData::forUnsubscribeRequest(client->counter().next(), subscriberId, topic, callback);
   
-  DuplexChannelPtr channel = client->getChannelForTopic(topic);
-  if (channel.get() == 0) {
-    LOG.errorStream() << "Trying to unsubscribe from (" << topic << ", " << subscriberId << ") but channel is dead";
-    callback->operationFailed(InvalidStateException());
-    return;
-  }
-  
-  doUnsubscribe(channel, data);  
+  DuplexChannelPtr channel = client->getChannel(topic);
+  doUnsubscribe(channel, data);
 }
 
 void SubscriberImpl::doUnsubscribe(const DuplexChannelPtr& channel, const PubSubDataPtr& data) {
@@ -293,10 +354,9 @@ void SubscriberImpl::doUnsubscribe(const DuplexChannelPtr& channel, const PubSub
 
 void SubscriberImpl::consume(const std::string& topic, const std::string& subscriberId, const MessageSeqId& messageSeqId) {
   TopicSubscriber t(topic, subscriberId);
-
-  topicsubscriber2handler_lock.lock();
+  
+  boost::shared_lock<boost::shared_mutex> lock(topicsubscriber2handler_lock);
   SubscriberClientChannelHandlerPtr handler = topicsubscriber2handler[t];
-  topicsubscriber2handler_lock.unlock();
 
   if (handler.get() == 0) {
     LOG.errorStream() << "Cannot consume. Bad handler for topic(" << topic << ") subscriberId(" << subscriberId << ") topicsubscriber2topic(" << &topicsubscriber2handler << ")";
@@ -309,16 +369,15 @@ void SubscriberImpl::consume(const std::string& topic, const std::string& subscr
   }
   
   PubSubDataPtr data = PubSubData::forConsumeRequest(client->counter().next(), subscriberId, topic, messageSeqId);  
-  OperationCallbackPtr writecb(new ConsumeWriteCallback(data));
+  OperationCallbackPtr writecb(new ConsumeWriteCallback(client, data));
   channel->writeRequest(data->getRequest(), writecb);
 }
 
 void SubscriberImpl::startDelivery(const std::string& topic, const std::string& subscriberId, const MessageHandlerCallbackPtr& callback) {
   TopicSubscriber t(topic, subscriberId);
 
-  topicsubscriber2handler_lock.lock();
+  boost::shared_lock<boost::shared_mutex> lock(topicsubscriber2handler_lock);
   SubscriberClientChannelHandlerPtr handler = topicsubscriber2handler[t];
-  topicsubscriber2handler_lock.unlock();
 
   if (handler.get() == 0) {
     LOG.errorStream() << "Trying to start deliver on a non existant handler topic = " << topic << ", subscriber = " << subscriberId;
@@ -328,10 +387,9 @@ void SubscriberImpl::startDelivery(const std::string& topic, const std::string&
 
 void SubscriberImpl::stopDelivery(const std::string& topic, const std::string& subscriberId) {
   TopicSubscriber t(topic, subscriberId);
-
-  topicsubscriber2handler_lock.lock();
+  
+  boost::shared_lock<boost::shared_mutex> lock(topicsubscriber2handler_lock);
   SubscriberClientChannelHandlerPtr handler = topicsubscriber2handler[t];
-  topicsubscriber2handler_lock.unlock();
 
   if (handler.get() == 0) {
     LOG.errorStream() << "Trying to start deliver on a non existant handler topic = " << topic << ", subscriber = " << subscriberId;
@@ -345,11 +403,14 @@ void SubscriberImpl::closeSubscription(const std::string& topic, const std::stri
   }
   TopicSubscriber t(topic, subscriberId);
 
-  topicsubscriber2handler_lock.lock();;
-  SubscriberClientChannelHandlerPtr handler = topicsubscriber2handler[t];
-  topicsubscriber2handler.erase(t);
-  topicsubscriber2handler_lock.unlock();;
-  if (handler) {
+  SubscriberClientChannelHandlerPtr handler;
+  {
+    boost::lock_guard<boost::shared_mutex> lock(topicsubscriber2handler_lock);
+    handler = topicsubscriber2handler[t];
+    topicsubscriber2handler.erase(t);
+  }
+  
+  if (handler.get() != 0) {
     handler->close();
   }
 }
@@ -357,16 +418,16 @@ void SubscriberImpl::closeSubscription(const std::string& topic, const std::stri
 /**
    takes ownership of txn
 */
-void SubscriberImpl::messageHandler(const PubSubResponse& m, const PubSubDataPtr& txn) {
+void SubscriberImpl::messageHandler(const PubSubResponsePtr& m, const PubSubDataPtr& txn) {
   if (!txn.get()) {
     LOG.errorStream() << "Invalid transaction";
     return;
   }
 
   if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "message received with status " << m.statuscode();
+    LOG.debugStream() << "message received with status " << m->statuscode();
   }
-  switch (m.statuscode()) {
+  switch (m->statuscode()) {
   case SUCCESS:
     txn->getCallback()->operationComplete();
     break;

+ 46 - 29
src/contrib/hedwig/client/src/main/cpp/lib/subscriberimpl.h

@@ -25,98 +25,115 @@
 #include <tr1/memory>
 #include <deque>
 
+#include <boost/shared_ptr.hpp>
+#include <boost/enable_shared_from_this.hpp>
+#include <boost/thread/shared_mutex.hpp>
+
 namespace Hedwig {
   class SubscriberWriteCallback : public OperationCallback {
   public:
-    SubscriberWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data);
+    SubscriberWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data);
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
   private:
-    ClientImplPtr client;
-    PubSubDataPtr data;
+    const ClientImplPtr client;
+    const PubSubDataPtr data;
   };
   
   class UnsubscribeWriteCallback : public OperationCallback {
   public:
-    UnsubscribeWriteCallback(ClientImplPtr& client, const PubSubDataPtr& data);
+    UnsubscribeWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data);
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
   private:
-    ClientImplPtr client;
-    PubSubDataPtr data;
+    const ClientImplPtr client;
+    const PubSubDataPtr data;
   };
 
   class ConsumeWriteCallback : public OperationCallback {
   public:
-    ConsumeWriteCallback(const PubSubDataPtr& data);
+    ConsumeWriteCallback(const ClientImplPtr& client, const PubSubDataPtr& data);
     ~ConsumeWriteCallback();
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
+    
+    static void timerComplete(const ClientImplPtr& client, const PubSubDataPtr& data, const boost::system::error_code& error);
   private:
-    PubSubDataPtr data;
+    const ClientImplPtr client;
+    const PubSubDataPtr data;
     };
 
   class SubscriberReconnectCallback : public OperationCallback {
   public: 
-    SubscriberReconnectCallback(ClientImplPtr& client, const PubSubDataPtr& origData);
+    SubscriberReconnectCallback(const ClientImplPtr& client, const PubSubDataPtr& origData);
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
   private:
-    ClientImplPtr client;
-    PubSubDataPtr origData;
+    const ClientImplPtr client;
+    const PubSubDataPtr origData;
   };
 
   class SubscriberClientChannelHandler;
-  typedef std::tr1::shared_ptr<SubscriberClientChannelHandler> SubscriberClientChannelHandlerPtr;
+  typedef boost::shared_ptr<SubscriberClientChannelHandler> SubscriberClientChannelHandlerPtr;
 
   class SubscriberConsumeCallback : public OperationCallback {
   public: 
-    SubscriberConsumeCallback(ClientImplPtr& client, const std::string& topic, const std::string& subscriberid, const MessageSeqId& msgid);
+    SubscriberConsumeCallback(const ClientImplPtr& client, const SubscriberClientChannelHandlerPtr& handler, const PubSubDataPtr& data, const PubSubResponsePtr& m);
 
     void operationComplete();
     void operationFailed(const std::exception& exception);
+    static void timerComplete(const SubscriberClientChannelHandlerPtr handler, 
+			      const PubSubResponsePtr m, 
+			      const boost::system::error_code& error);
+
   private:
-    ClientImplPtr client;
-    const std::string topic;
-    const std::string subscriberid;
-    MessageSeqId msgid;
+    const ClientImplPtr client;
+    const SubscriberClientChannelHandlerPtr handler;
+    
+    const PubSubDataPtr data;
+    const PubSubResponsePtr m;
   };
 
-  class SubscriberClientChannelHandler : public HedwigClientChannelHandler {
+  class SubscriberClientChannelHandler : public HedwigClientChannelHandler, 
+					 public boost::enable_shared_from_this<SubscriberClientChannelHandler> {
   public: 
-    SubscriberClientChannelHandler(ClientImplPtr& client, SubscriberImpl& subscriber, const PubSubDataPtr& data);
+    SubscriberClientChannelHandler(const ClientImplPtr& client, SubscriberImpl& subscriber, const PubSubDataPtr& data);
     ~SubscriberClientChannelHandler();
 
-    void messageReceived(DuplexChannel* channel, const PubSubResponse& m);
-    void channelDisconnected(DuplexChannel* channel, const std::exception& e);
+    void messageReceived(const DuplexChannelPtr& channel, const PubSubResponsePtr& m);
+    void channelDisconnected(const DuplexChannelPtr& channel, const std::exception& e);
 
     void startDelivery(const MessageHandlerCallbackPtr& handler);
     void stopDelivery();
 
-    void handoverDelivery(SubscriberClientChannelHandler* newHandler);
+    void handoverDelivery(const SubscriberClientChannelHandlerPtr& newHandler);
 
     void setChannel(const DuplexChannelPtr& channel);
     DuplexChannelPtr& getChannel();
 
+    static void reconnectTimerComplete(const SubscriberClientChannelHandlerPtr handler, const DuplexChannelPtr channel, const std::exception e, 
+				       const boost::system::error_code& error);
+
     void close();
   private:
 
     SubscriberImpl& subscriber;
-#warning "put some limit on this to stop it growing forever"
-    std::deque<Message> queue;
+    std::deque<PubSubResponsePtr> queue;
+    
     MessageHandlerCallbackPtr handler;
     PubSubDataPtr origData;
     DuplexChannelPtr channel;
     bool closed;
+    bool should_wait;
   };
 
   class SubscriberImpl : public Subscriber {
   public:
-    SubscriberImpl(ClientImplPtr& client);
+    SubscriberImpl(const ClientImplPtr& client);
     ~SubscriberImpl();
 
     void subscribe(const std::string& topic, const std::string& subscriberId, const SubscribeRequest::CreateOrAttach mode);
@@ -132,16 +149,16 @@ namespace Hedwig {
 
     void closeSubscription(const std::string& topic, const std::string& subscriberId);
 
-    void messageHandler(const PubSubResponse& m, const PubSubDataPtr& txn);
+    void messageHandler(const PubSubResponsePtr& m, const PubSubDataPtr& txn);
 
     void doSubscribe(const DuplexChannelPtr& channel, const PubSubDataPtr& data, const SubscriberClientChannelHandlerPtr& handler);
     void doUnsubscribe(const DuplexChannelPtr& channel, const PubSubDataPtr& data);
 
   private:
-    ClientImplPtr client;
+    const ClientImplPtr client;
     
-    std::tr1::unordered_map<TopicSubscriber, SubscriberClientChannelHandlerPtr> topicsubscriber2handler;
-    Mutex topicsubscriber2handler_lock;	    
+    std::tr1::unordered_map<TopicSubscriber, SubscriberClientChannelHandlerPtr, TopicSubscriberHash > topicsubscriber2handler;
+    boost::shared_mutex topicsubscriber2handler_lock;	    
   };
 
 };

+ 0 - 92
src/contrib/hedwig/client/src/main/cpp/lib/util.cpp

@@ -139,95 +139,3 @@ HostAddress HostAddress::fromString(std::string str) {
   return h;
 }
 
-WaitConditionBase::WaitConditionBase() {
-  pthread_mutex_init(&mutex, NULL);
-  pthread_cond_init(&cond, NULL);  
-}
-
-WaitConditionBase::~WaitConditionBase() {
-  pthread_mutex_destroy(&mutex);
-  pthread_cond_destroy(&cond);
-}
-    
-void WaitConditionBase::wait() {
-  pthread_mutex_lock(&mutex);
-  while (!isTrue()) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debugStream() << "wait: condition is false for " << this;
-    }
-
-    pthread_cond_wait(&cond, &mutex); 
-  }
-  pthread_mutex_unlock(&mutex);
-}
-
-void WaitConditionBase::lock() {
-  pthread_mutex_lock(&mutex);
-}
-
-void WaitConditionBase::signalAndUnlock() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "signal: signal " << this;
-  }
-  
-  pthread_cond_signal(&cond);
-  
-  pthread_mutex_unlock(&mutex);
-}
-
-Mutex::Mutex() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Creating mutex " << this;
-  }
-  int error = pthread_mutex_init(&mutex, NULL);
-  if (error != 0) {
-    LOG.errorStream() << "Error initiating mutex " << error;
-  }
-}
-
-Mutex::~Mutex() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Destroying mutex " << this;
-  }
-
-  int error = pthread_mutex_destroy(&mutex);
-  if (error != 0) {
-    LOG.errorStream() << "Error destroying mutex " << this << " " << error;
-  }
-}
-
-void Mutex::lock() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Locking mutex " << this;
-  }
-    
-  int error = pthread_mutex_lock(&mutex);
-  if (error != 0) {
-    LOG.errorStream() << "Error locking mutex " << this << " " << error;
-  }
-}
-
-void Mutex::unlock() {
-  if (LOG.isDebugEnabled()) {
-    LOG.debugStream() << "Unlocking mutex " << this;
-  }
-
-  int error = pthread_mutex_unlock(&mutex);
-  if (error != 0) {
-    LOG.errorStream() << "Error unlocking mutex " << this << " " << error;
-  }
-}
-
-std::size_t std::tr1::hash<HostAddress>::operator()(const HostAddress& address) const {
-  return (address.ip() << 16) & (address.port());
-}
-
-std::size_t std::tr1::hash<DuplexChannel*>::operator()(const DuplexChannel* channel) const {
-  return reinterpret_cast<std::size_t>(channel);
-}
-
-std::size_t std::tr1::hash<TopicSubscriber>::operator()(const TopicSubscriber& topicsub) const {
-  std::string fullstr = topicsub.first + topicsub.second;
-  return std::tr1::hash<std::string>()(fullstr);
-}
-

+ 11 - 47
src/contrib/hedwig/client/src/main/cpp/lib/util.h

@@ -60,63 +60,27 @@ namespace Hedwig {
     struct sockaddr_in socket_addr;
   };
 
-  class DuplexChannel;  
-  
-  class Mutex {
-  public:
-    Mutex();
-    ~Mutex();
-    
-    void lock();
-    void unlock();
-  private:
-    pthread_mutex_t mutex;
-  };
-
-  class WaitConditionBase {
-  public:
-    WaitConditionBase();
-    virtual ~WaitConditionBase();
-    
-    void wait(); 
-    void lock();
-    void signalAndUnlock();
-
-    virtual bool isTrue() = 0;
-  private:
-
-    pthread_mutex_t mutex;
-    pthread_cond_t cond;    
-  };
-
-};
-
-namespace std 
-{
-  namespace tr1 
-  {
   /**
      Hash a host address. Takes the least significant 16-bits of the address and the 16-bits of the
      port and packs them into one 32-bit number. While collisons are theoretically very possible, they
      shouldn't happen as the hedwig servers should be in the same subnet.
   */
-  template <> struct hash<Hedwig::HostAddress> : public unary_function<Hedwig::HostAddress, size_t> {
-    size_t operator()(const Hedwig::HostAddress& address) const;
+  struct HostAddressHash : public std::unary_function<Hedwig::HostAddress, size_t> {
+    size_t operator()(const Hedwig::HostAddress& address) const {
+        return (address.ip() << 16) & (address.port());
+    }
   };
 
-  /**
-     Hash a channel pointer, just returns the pointer.
-  */
-  template <> struct hash<Hedwig::DuplexChannel*> : public unary_function<Hedwig::DuplexChannel*, size_t> {
-    size_t operator()(const Hedwig::DuplexChannel* channel) const;
-  };
 
   /**
      Hash a channel pointer, just returns the pointer.
   */
-  template <> struct hash<Hedwig::TopicSubscriber> : public unary_function<Hedwig::TopicSubscriber, size_t> {
-    size_t operator()(const Hedwig::TopicSubscriber& topicsub) const;
+  struct TopicSubscriberHash : public std::unary_function<Hedwig::TopicSubscriber, size_t> {
+    size_t operator()(const Hedwig::TopicSubscriber& topicsub) const {
+      std::string fullstr = topicsub.first + topicsub.second;
+      return std::tr1::hash<std::string>()(fullstr);
+    }
   };
-  }
-}
+};
+
 #endif

+ 111 - 0
src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_asio.m4

@@ -0,0 +1,111 @@
+# ===========================================================================
+#       http://www.gnu.org/software/autoconf-archive/ax_boost_asio.html
+# ===========================================================================
+#
+# SYNOPSIS
+#
+#   AX_BOOST_ASIO
+#
+# DESCRIPTION
+#
+#   Test for Asio library from the Boost C++ libraries. The macro requires a
+#   preceding call to AX_BOOST_BASE. Further documentation is available at
+#   <http://randspringer.de/boost/index.html>.
+#
+#   This macro calls:
+#
+#     AC_SUBST(BOOST_ASIO_LIB)
+#
+#   And sets:
+#
+#     HAVE_BOOST_ASIO
+#
+# LICENSE
+#
+#   Copyright (c) 2008 Thomas Porschberg <thomas@randspringer.de>
+#   Copyright (c) 2008 Pete Greenwell <pete@mu.org>
+#
+#   Copying and distribution of this file, with or without modification, are
+#   permitted in any medium without royalty provided the copyright notice
+#   and this notice are preserved. This file is offered as-is, without any
+#   warranty.
+
+#serial 9
+
+AC_DEFUN([AX_BOOST_ASIO],
+[
+	AC_ARG_WITH([boost-asio],
+	AS_HELP_STRING([--with-boost-asio@<:@=special-lib@:>@],
+                   [use the ASIO library from boost - it is possible to specify a certain library for the linker
+                        e.g. --with-boost-asio=boost_system-gcc41-mt-1_34 ]),
+        [
+        if test "$withval" = "no"; then
+			want_boost="no"
+        elif test "$withval" = "yes"; then
+            want_boost="yes"
+            ax_boost_user_asio_lib=""
+        else
+		    want_boost="yes"
+        	ax_boost_user_asio_lib="$withval"
+		fi
+        ],
+        [want_boost="yes"]
+	)
+
+	if test "x$want_boost" = "xyes"; then
+        AC_REQUIRE([AC_PROG_CC])
+		CPPFLAGS_SAVED="$CPPFLAGS"
+		CPPFLAGS="$CPPFLAGS $BOOST_CPPFLAGS"
+		export CPPFLAGS
+
+		LDFLAGS_SAVED="$LDFLAGS"
+		LDFLAGS="$LDFLAGS $BOOST_LDFLAGS"
+		export LDFLAGS
+
+        AC_CACHE_CHECK(whether the Boost::ASIO library is available,
+					   ax_cv_boost_asio,
+        [AC_LANG_PUSH([C++])
+		 AC_COMPILE_IFELSE(AC_LANG_PROGRAM([[ @%:@include <boost/asio.hpp>
+											]],
+                                  [[
+
+                                    boost::asio::io_service io;
+                                    boost::system::error_code timer_result;
+                                    boost::asio::deadline_timer t(io);
+                                    t.cancel();
+                                    io.run_one();
+									return 0;
+                                   ]]),
+                             ax_cv_boost_asio=yes, ax_cv_boost_asio=no)
+         AC_LANG_POP([C++])
+		])
+		if test "x$ax_cv_boost_asio" = "xyes"; then
+			AC_DEFINE(HAVE_BOOST_ASIO,,[define if the Boost::ASIO library is available])
+			BN=boost_system
+            if test "x$ax_boost_user_asio_lib" = "x"; then
+				for ax_lib in $BN $BN-$CC $BN-$CC-mt $BN-$CC-mt-s $BN-$CC-s \
+                              lib$BN lib$BN-$CC lib$BN-$CC-mt lib$BN-$CC-mt-s lib$BN-$CC-s \
+                              $BN-mgw $BN-mgw $BN-mgw-mt $BN-mgw-mt-s $BN-mgw-s ; do
+				    AC_CHECK_LIB($ax_lib, main, [BOOST_ASIO_LIB="-l$ax_lib" AC_SUBST(BOOST_ASIO_LIB) link_thread="yes" break],
+                                 [link_thread="no"])
+  				done
+            else
+               for ax_lib in $ax_boost_user_asio_lib $BN-$ax_boost_user_asio_lib; do
+				      AC_CHECK_LIB($ax_lib, main,
+                                   [BOOST_ASIO_LIB="-l$ax_lib" AC_SUBST(BOOST_ASIO_LIB) link_asio="yes" break],
+                                   [link_asio="no"])
+                  done
+
+            fi
+            if test "x$ax_lib" = "x"; then
+                AC_MSG_ERROR(Could not find a version of the library!)
+            fi
+			if test "x$link_asio" = "xno"; then
+				AC_MSG_ERROR(Could not link against $ax_lib !)
+			fi
+		fi
+
+		CPPFLAGS="$CPPFLAGS_SAVED"
+    	LDFLAGS="$LDFLAGS_SAVED"
+	fi
+])

+ 252 - 0
src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_base.m4

@@ -0,0 +1,252 @@
+# ===========================================================================
+#       http://www.gnu.org/software/autoconf-archive/ax_boost_base.html
+# ===========================================================================
+#
+# SYNOPSIS
+#
+#   AX_BOOST_BASE([MINIMUM-VERSION], [ACTION-IF-FOUND], [ACTION-IF-NOT-FOUND])
+#
+# DESCRIPTION
+#
+#   Test for the Boost C++ libraries of a particular version (or newer)
+#
+#   If no path to the installed boost library is given the macro searchs
+#   under /usr, /usr/local, /opt and /opt/local and evaluates the
+#   $BOOST_ROOT environment variable. Further documentation is available at
+#   <http://randspringer.de/boost/index.html>.
+#
+#   This macro calls:
+#
+#     AC_SUBST(BOOST_CPPFLAGS) / AC_SUBST(BOOST_LDFLAGS)
+#
+#   And sets:
+#
+#     HAVE_BOOST
+#
+# LICENSE
+#
+#   Copyright (c) 2008 Thomas Porschberg <thomas@randspringer.de>
+#   Copyright (c) 2009 Peter Adolphs
+#
+#   Copying and distribution of this file, with or without modification, are
+#   permitted in any medium without royalty provided the copyright notice
+#   and this notice are preserved. This file is offered as-is, without any
+#   warranty.
+
+#serial 17
+
+AC_DEFUN([AX_BOOST_BASE],
+[
+AC_ARG_WITH([boost],
+  [AS_HELP_STRING([--with-boost@<:@=ARG@:>@],
+    [use Boost library from a standard location (ARG=yes),
+     from the specified location (ARG=<path>),
+     or disable it (ARG=no)
+     @<:@ARG=yes@:>@ ])],
+    [
+    if test "$withval" = "no"; then
+        want_boost="no"
+    elif test "$withval" = "yes"; then
+        want_boost="yes"
+        ac_boost_path=""
+    else
+        want_boost="yes"
+        ac_boost_path="$withval"
+    fi
+    ],
+    [want_boost="yes"])
+
+
+AC_ARG_WITH([boost-libdir],
+        AS_HELP_STRING([--with-boost-libdir=LIB_DIR],
+        [Force given directory for boost libraries. Note that this will overwrite library path detection, so use this parameter only if default library detection fails and you know exactly where your boost libraries are located.]),
+        [
+        if test -d "$withval"
+        then
+                ac_boost_lib_path="$withval"
+        else
+                AC_MSG_ERROR(--with-boost-libdir expected directory name)
+        fi
+        ],
+        [ac_boost_lib_path=""]
+)
+
+if test "x$want_boost" = "xyes"; then
+    boost_lib_version_req=ifelse([$1], ,1.20.0,$1)
+    boost_lib_version_req_shorten=`expr $boost_lib_version_req : '\([[0-9]]*\.[[0-9]]*\)'`
+    boost_lib_version_req_major=`expr $boost_lib_version_req : '\([[0-9]]*\)'`
+    boost_lib_version_req_minor=`expr $boost_lib_version_req : '[[0-9]]*\.\([[0-9]]*\)'`
+    boost_lib_version_req_sub_minor=`expr $boost_lib_version_req : '[[0-9]]*\.[[0-9]]*\.\([[0-9]]*\)'`
+    if test "x$boost_lib_version_req_sub_minor" = "x" ; then
+        boost_lib_version_req_sub_minor="0"
+        fi
+    WANT_BOOST_VERSION=`expr $boost_lib_version_req_major \* 100000 \+  $boost_lib_version_req_minor \* 100 \+ $boost_lib_version_req_sub_minor`
+    AC_MSG_CHECKING(for boostlib >= $boost_lib_version_req)
+    succeeded=no
+
+    dnl On x86_64 systems check for system libraries in both lib64 and lib.
+    dnl The former is specified by FHS, but e.g. Debian does not adhere to
+    dnl this (as it rises problems for generic multi-arch support).
+    dnl The last entry in the list is chosen by default when no libraries
+    dnl are found, e.g. when only header-only libraries are installed!
+    libsubdirs="lib"
+    if test `uname -m` = x86_64; then
+        libsubdirs="lib64 lib lib64"
+    fi
+
+    dnl first we check the system location for boost libraries
+    dnl this location ist chosen if boost libraries are installed with the --layout=system option
+    dnl or if you install boost with RPM
+    if test "$ac_boost_path" != ""; then
+        BOOST_LDFLAGS="-L$ac_boost_path/$libsubdir"
+        BOOST_CPPFLAGS="-I$ac_boost_path/include"
+    elif test "$cross_compiling" != yes; then
+        for ac_boost_path_tmp in /usr /usr/local /opt /opt/local ; do
+            if test -d "$ac_boost_path_tmp/include/boost" && test -r "$ac_boost_path_tmp/include/boost"; then
+                for libsubdir in $libsubdirs ; do
+                    if ls "$ac_boost_path_tmp/$libsubdir/libboost_"* >/dev/null 2>&1 ; then break; fi
+                done
+                BOOST_LDFLAGS="-L$ac_boost_path_tmp/$libsubdir"
+                BOOST_CPPFLAGS="-I$ac_boost_path_tmp/include"
+                break;
+            fi
+        done
+    fi
+
+    dnl overwrite ld flags if we have required special directory with
+    dnl --with-boost-libdir parameter
+    if test "$ac_boost_lib_path" != ""; then
+       BOOST_LDFLAGS="-L$ac_boost_lib_path"
+    fi
+
+    CPPFLAGS_SAVED="$CPPFLAGS"
+    CPPFLAGS="$CPPFLAGS $BOOST_CPPFLAGS"
+    export CPPFLAGS
+
+    LDFLAGS_SAVED="$LDFLAGS"
+    LDFLAGS="$LDFLAGS $BOOST_LDFLAGS"
+    export LDFLAGS
+
+    AC_REQUIRE([AC_PROG_CXX])
+    AC_LANG_PUSH(C++)
+        AC_COMPILE_IFELSE([AC_LANG_PROGRAM([[
+    @%:@include <boost/version.hpp>
+    ]], [[
+    #if BOOST_VERSION >= $WANT_BOOST_VERSION
+    // Everything is okay
+    #else
+    #  error Boost version is too old
+    #endif
+    ]])],[
+        AC_MSG_RESULT(yes)
+    succeeded=yes
+    found_system=yes
+        ],[
+        ])
+    AC_LANG_POP([C++])
+
+
+
+    dnl if we found no boost with system layout we search for boost libraries
+    dnl built and installed without the --layout=system option or for a staged(not installed) version
+    if test "x$succeeded" != "xyes"; then
+        _version=0
+        if test "$ac_boost_path" != ""; then
+            if test -d "$ac_boost_path" && test -r "$ac_boost_path"; then
+                for i in `ls -d $ac_boost_path/include/boost-* 2>/dev/null`; do
+                    _version_tmp=`echo $i | sed "s#$ac_boost_path##" | sed 's/\/include\/boost-//' | sed 's/_/./'`
+                    V_CHECK=`expr $_version_tmp \> $_version`
+                    if test "$V_CHECK" = "1" ; then
+                        _version=$_version_tmp
+                    fi
+                    VERSION_UNDERSCORE=`echo $_version | sed 's/\./_/'`
+                    BOOST_CPPFLAGS="-I$ac_boost_path/include/boost-$VERSION_UNDERSCORE"
+                done
+            fi
+        else
+            if test "$cross_compiling" != yes; then
+                for ac_boost_path in /usr /usr/local /opt /opt/local ; do
+                    if test -d "$ac_boost_path" && test -r "$ac_boost_path"; then
+                        for i in `ls -d $ac_boost_path/include/boost-* 2>/dev/null`; do
+                            _version_tmp=`echo $i | sed "s#$ac_boost_path##" | sed 's/\/include\/boost-//' | sed 's/_/./'`
+                            V_CHECK=`expr $_version_tmp \> $_version`
+                            if test "$V_CHECK" = "1" ; then
+                                _version=$_version_tmp
+                                best_path=$ac_boost_path
+                            fi
+                        done
+                    fi
+                done
+
+                VERSION_UNDERSCORE=`echo $_version | sed 's/\./_/'`
+                BOOST_CPPFLAGS="-I$best_path/include/boost-$VERSION_UNDERSCORE"
+                if test "$ac_boost_lib_path" = ""; then
+                    for libsubdir in $libsubdirs ; do
+                        if ls "$best_path/$libsubdir/libboost_"* >/dev/null 2>&1 ; then break; fi
+                    done
+                    BOOST_LDFLAGS="-L$best_path/$libsubdir"
+                fi
+            fi
+
+            if test "x$BOOST_ROOT" != "x"; then
+                for libsubdir in $libsubdirs ; do
+                    if ls "$BOOST_ROOT/stage/$libsubdir/libboost_"* >/dev/null 2>&1 ; then break; fi
+                done
+                if test -d "$BOOST_ROOT" && test -r "$BOOST_ROOT" && test -d "$BOOST_ROOT/stage/$libsubdir" && test -r "$BOOST_ROOT/stage/$libsubdir"; then
+                    version_dir=`expr //$BOOST_ROOT : '.*/\(.*\)'`
+                    stage_version=`echo $version_dir | sed 's/boost_//' | sed 's/_/./g'`
+                        stage_version_shorten=`expr $stage_version : '\([[0-9]]*\.[[0-9]]*\)'`
+                    V_CHECK=`expr $stage_version_shorten \>\= $_version`
+                    if test "$V_CHECK" = "1" -a "$ac_boost_lib_path" = "" ; then
+                        AC_MSG_NOTICE(We will use a staged boost library from $BOOST_ROOT)
+                        BOOST_CPPFLAGS="-I$BOOST_ROOT"
+                        BOOST_LDFLAGS="-L$BOOST_ROOT/stage/$libsubdir"
+                    fi
+                fi
+            fi
+        fi
+
+        CPPFLAGS="$CPPFLAGS $BOOST_CPPFLAGS"
+        export CPPFLAGS
+        LDFLAGS="$LDFLAGS $BOOST_LDFLAGS"
+        export LDFLAGS
+
+        AC_LANG_PUSH(C++)
+            AC_COMPILE_IFELSE([AC_LANG_PROGRAM([[
+        @%:@include <boost/version.hpp>
+        ]], [[
+        #if BOOST_VERSION >= $WANT_BOOST_VERSION
+        // Everything is okay
+        #else
+        #  error Boost version is too old
+        #endif
+        ]])],[
+            AC_MSG_RESULT(yes)
+        succeeded=yes
+        found_system=yes
+            ],[
+            ])
+        AC_LANG_POP([C++])
+    fi
+
+    if test "$succeeded" != "yes" ; then
+        if test "$_version" = "0" ; then
+            AC_MSG_NOTICE([[We could not detect the boost libraries (version $boost_lib_version_req_shorten or higher). If you have a staged boost library (still not installed) please specify \$BOOST_ROOT in your environment and do not give a PATH to --with-boost option.  If you are sure you have boost installed, then check your version number looking in <boost/version.hpp>. See http://randspringer.de/boost for more documentation.]])
+        else
+            AC_MSG_NOTICE([Your boost libraries seems to old (version $_version).])
+        fi
+        # execute ACTION-IF-NOT-FOUND (if present):
+        ifelse([$3], , :, [$3])
+    else
+        AC_SUBST(BOOST_CPPFLAGS)
+        AC_SUBST(BOOST_LDFLAGS)
+        AC_DEFINE(HAVE_BOOST,,[define if the Boost library is available])
+        # execute ACTION-IF-FOUND (if present):
+        ifelse([$2], , :, [$2])
+    fi
+
+    CPPFLAGS="$CPPFLAGS_SAVED"
+    LDFLAGS="$LDFLAGS_SAVED"
+fi
+
+])

+ 149 - 0
src/contrib/hedwig/client/src/main/cpp/m4/ax_boost_thread.m4

@@ -0,0 +1,149 @@
+# ===========================================================================
+#      http://www.gnu.org/software/autoconf-archive/ax_boost_thread.html
+# ===========================================================================
+#
+# SYNOPSIS
+#
+#   AX_BOOST_THREAD
+#
+# DESCRIPTION
+#
+#   Test for Thread library from the Boost C++ libraries. The macro requires
+#   a preceding call to AX_BOOST_BASE. Further documentation is available at
+#   <http://randspringer.de/boost/index.html>.
+#
+#   This macro calls:
+#
+#     AC_SUBST(BOOST_THREAD_LIB)
+#
+#   And sets:
+#
+#     HAVE_BOOST_THREAD
+#
+# LICENSE
+#
+#   Copyright (c) 2009 Thomas Porschberg <thomas@randspringer.de>
+#   Copyright (c) 2009 Michael Tindal
+#
+#   Copying and distribution of this file, with or without modification, are
+#   permitted in any medium without royalty provided the copyright notice
+#   and this notice are preserved. This file is offered as-is, without any
+#   warranty.
+
+#serial 17
+
+AC_DEFUN([AX_BOOST_THREAD],
+[
+	AC_ARG_WITH([boost-thread],
+	AS_HELP_STRING([--with-boost-thread@<:@=special-lib@:>@],
+                   [use the Thread library from boost - it is possible to specify a certain library for the linker
+                        e.g. --with-boost-thread=boost_thread-gcc-mt ]),
+        [
+        if test "$withval" = "no"; then
+			want_boost="no"
+        elif test "$withval" = "yes"; then
+            want_boost="yes"
+            ax_boost_user_thread_lib=""
+        else
+		    want_boost="yes"
+        	ax_boost_user_thread_lib="$withval"
+		fi
+        ],
+        [want_boost="yes"]
+	)
+
+	if test "x$want_boost" = "xyes"; then
+        AC_REQUIRE([AC_PROG_CC])
+        AC_REQUIRE([AC_CANONICAL_BUILD])
+		CPPFLAGS_SAVED="$CPPFLAGS"
+		CPPFLAGS="$CPPFLAGS $BOOST_CPPFLAGS"
+		export CPPFLAGS
+
+		LDFLAGS_SAVED="$LDFLAGS"
+		LDFLAGS="$LDFLAGS $BOOST_LDFLAGS"
+		export LDFLAGS
+
+        AC_CACHE_CHECK(whether the Boost::Thread library is available,
+					   ax_cv_boost_thread,
+        [AC_LANG_PUSH([C++])
+			 CXXFLAGS_SAVE=$CXXFLAGS
+
+			 if test "x$build_os" = "xsolaris" ; then
+  				 CXXFLAGS="-pthreads $CXXFLAGS"
+			 elif test "x$build_os" = "xming32" ; then
+				 CXXFLAGS="-mthreads $CXXFLAGS"
+			 else
+				CXXFLAGS="-pthread $CXXFLAGS"
+			 fi
+			 AC_COMPILE_IFELSE(AC_LANG_PROGRAM([[@%:@include <boost/thread/thread.hpp>]],
+                                   [[boost::thread_group thrds;
+                                   return 0;]]),
+                   ax_cv_boost_thread=yes, ax_cv_boost_thread=no)
+			 CXXFLAGS=$CXXFLAGS_SAVE
+             AC_LANG_POP([C++])
+		])
+		if test "x$ax_cv_boost_thread" = "xyes"; then
+           if test "x$build_os" = "xsolaris" ; then
+			  BOOST_CPPFLAGS="-pthreads $BOOST_CPPFLAGS"
+		   elif test "x$build_os" = "xming32" ; then
+			  BOOST_CPPFLAGS="-mthreads $BOOST_CPPFLAGS"
+		   else
+			  BOOST_CPPFLAGS="-pthread $BOOST_CPPFLAGS"
+		   fi
+
+			AC_SUBST(BOOST_CPPFLAGS)
+
+			AC_DEFINE(HAVE_BOOST_THREAD,,[define if the Boost::Thread library is available])
+            BOOSTLIBDIR=`echo $BOOST_LDFLAGS | sed -e 's/@<:@^\/@:>@*//'`
+
+			LDFLAGS_SAVE=$LDFLAGS
+                        case "x$build_os" in
+                          *bsd* )
+                               LDFLAGS="-pthread $LDFLAGS"
+                          break;
+                          ;;
+                        esac
+            if test "x$ax_boost_user_thread_lib" = "x"; then
+                for libextension in `ls $BOOSTLIBDIR/libboost_thread*.so* 2>/dev/null | sed 's,.*/,,' | sed -e 's;^lib\(boost_thread.*\)\.so.*$;\1;'` `ls $BOOSTLIBDIR/libboost_thread*.a* 2>/dev/null | sed 's,.*/,,' | sed -e 's;^lib\(boost_thread.*\)\.a*$;\1;'`; do
+                     ax_lib=${libextension}
+				    AC_CHECK_LIB($ax_lib, exit,
+                                 [BOOST_THREAD_LIB="-l$ax_lib"; AC_SUBST(BOOST_THREAD_LIB) link_thread="yes"; break],
+                                 [link_thread="no"])
+  				done
+                if test "x$link_thread" != "xyes"; then
+                for libextension in `ls $BOOSTLIBDIR/boost_thread*.dll* 2>/dev/null | sed 's,.*/,,' | sed -e 's;^\(boost_thread.*\)\.dll.*$;\1;'` `ls $BOOSTLIBDIR/boost_thread*.a* 2>/dev/null | sed 's,.*/,,' | sed -e 's;^\(boost_thread.*\)\.a*$;\1;'` ; do
+                     ax_lib=${libextension}
+				    AC_CHECK_LIB($ax_lib, exit,
+                                 [BOOST_THREAD_LIB="-l$ax_lib"; AC_SUBST(BOOST_THREAD_LIB) link_thread="yes"; break],
+                                 [link_thread="no"])
+  				done
+                fi
+
+            else
+               for ax_lib in $ax_boost_user_thread_lib boost_thread-$ax_boost_user_thread_lib; do
+				      AC_CHECK_LIB($ax_lib, exit,
+                                   [BOOST_THREAD_LIB="-l$ax_lib"; AC_SUBST(BOOST_THREAD_LIB) link_thread="yes"; break],
+                                   [link_thread="no"])
+                  done
+
+            fi
+            if test "x$ax_lib" = "x"; then
+                AC_MSG_ERROR(Could not find a version of the library!)
+            fi
+			if test "x$link_thread" = "xno"; then
+				AC_MSG_ERROR(Could not link against $ax_lib !)
+                        else
+                           case "x$build_os" in
+                              *bsd* )
+			        BOOST_LDFLAGS="-pthread $BOOST_LDFLAGS"
+                              break;
+                              ;;
+                           esac
+
+			fi
+		fi
+
+		CPPFLAGS="$CPPFLAGS_SAVED"
+    	LDFLAGS="$LDFLAGS_SAVED"
+	fi
+])

+ 49 - 0
src/contrib/hedwig/client/src/main/cpp/scripts/log4cpp.conf

@@ -0,0 +1,49 @@
+#
+# 
+# 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.
+# 
+#
+
+log4j.appender.rootAppender=org.apache.log4j.ConsoleAppender
+log4j.appender.rootAppender.layout=org.apache.log4j.BasicLayout
+
+#log4j.appender.hedwig=org.apache.log4j.RollingFileAppender
+log4j.appender.hedwig=org.apache.log4j.ConsoleAppender
+#log4j.appender.hedwig.fileName=./testLog.log
+log4j.appender.hedwig.layout=org.apache.log4j.PatternLayout
+log4j.appender.hedwig.layout.ConversionPattern=[%d{%H:%M:%S.%l}] %t %c %p - %m%n
+log4j.appender.hedwig.layout=org.apache.log4j.PatternLayout
+log4j.appender.hedwig.layout.ConversionPattern=%.5m%n
+
+log4j.appender.hedwigtest=org.apache.log4j.ConsoleAppender
+#log4j.appender.hedwig.fileName=./testLog.log
+log4j.appender.hedwigtest.layout=org.apache.log4j.PatternLayout
+log4j.appender.hedwigtest.layout.ConversionPattern=[%d{%H:%M:%S.%l}] %c %p - %m%n
+log4j.appender.hedwigtest.layout=org.apache.log4j.PatternLayout
+log4j.appender.hedwigtest.layout.ConversionPattern=%.5m%n
+
+# category
+log4j.category.hedwig=DEBUG, hedwig
+log4j.rootCategory=DEBUG
+
+#log4j.category.hedwig.channel=ERROR
+log4j.category.hedwig.util=ERROR
+log4j.category.hedwigtest.servercontrol=ERROR
+
+log4j.category.hedwigtest=DEBUG, hedwigtest
+log4j.rootCategory=DEBUG

+ 64 - 0
src/contrib/hedwig/client/src/main/cpp/scripts/network-delays.sh

@@ -0,0 +1,64 @@
+#!/bin/bash
+#
+# 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.
+#
+
+setup_delays() {
+
+    UNAME=`uname -s`
+
+    echo "Setting delay to ${1}ms"
+    case "$UNAME" in
+	Darwin|FreeBSD)
+	    sudo ipfw pipe 1 config delay ${1}ms
+	    sudo ipfw add pipe 1 dst-port 12349 
+	    sudo ipfw add pipe 1 dst-port 12350
+	    sudo ipfw add pipe 1 src-port 12349 
+	    sudo ipfw add pipe 1 src-port 12350 
+            ;;
+	Linux)
+	    sudo tc qdisc add dev lo root handle 1: prio
+	    sudo tc qdisc add dev lo parent 1:3 handle 30: netem delay ${1}ms 
+	    sudo tc filter add dev lo protocol ip parent 1:0 prio 3 u32 match ip dport 12349 0xffff flowid 1:3
+	    sudo tc filter add dev lo protocol ip parent 1:0 prio 3 u32 match ip dport 12350 0xffff flowid 1:3
+	    sudo tc filter add dev lo protocol ip parent 1:0 prio 3 u32 match ip sport 12349 0xffff flowid 1:3
+	    sudo tc filter add dev lo protocol ip parent 1:0 prio 3 u32 match ip sport 12350 0xffff flowid 1:3
+	    ;;
+	*)
+	    echo "Unknown system type, $UNAME, only Linux, Darwin & FreeBSD supported"
+	    ;;
+    esac
+}
+
+clear_delays() {
+    UNAME=`uname -s`
+
+    case "$UNAME" in
+	Darwin|FreeBSD)
+	    echo "Flushing ipfw"
+	    sudo ipfw -f -q flush
+            ;;
+	Linux)
+	    echo "Clearing delay"
+	    sudo tc qdisc del dev lo root
+	    ;;
+	*)
+	    echo "Unknown system type, $UNAME, only Linux, Darwin & FreeBSD supported"
+	    ;;
+    esac
+}
+

+ 49 - 0
src/contrib/hedwig/client/src/main/cpp/scripts/server-control.sh

@@ -0,0 +1,49 @@
+#!/bin/bash
+#
+# 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.
+#
+
+HEDWIGBASE=../../../../..
+
+HEDWIGJAR=`ls $HEDWIGBASE/server/target/server-*-with-dependencies.jar`
+if [ ! $? -eq 0 ]; then
+    echo "\n\nCould not find server-VERSION-with-dependencies.jar. \nYou need to build the java part of hedwig. \nRun mvn package in the toplevel hedwig directory.\n\n"
+    exit 1;
+fi
+
+HEDWIGSERVERTESTS=$HEDWIGBASE/server/target/test-classes/
+if [ ! -e $HEDWIGSERVERTESTS ]; then
+    echo "\n\nThe hedwig java server tests need to be build.\b\b"
+    exit 1;
+fi
+
+export CP=.:$HEDWIGJAR:$HEDWIGSERVERTESTS
+
+start_control_server() {
+    if [ -e server-control.pid ]; then
+	kill -9 `cat server-control.pid`
+	rm server-control.pid
+    fi
+    java -cp $CP  -Dlog4j.configuration=log4j.properties org.apache.hedwig.ServerControlDaemon  <&-  1> servercontrol.out  2>&1  &
+    echo $! > server-control.pid
+    sleep 5
+}
+
+stop_control_server() {
+    kill -9 `cat server-control.pid`
+    rm server-control.pid
+}

+ 95 - 0
src/contrib/hedwig/client/src/main/cpp/scripts/tester.sh

@@ -0,0 +1,95 @@
+#!/bin/bash
+#
+# 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.
+#
+
+cd `dirname $0`;
+
+source network-delays.sh
+source server-control.sh
+
+all() {
+    if [ "z$HEDWIG_NETWORK_DELAY" != "z" ]; then
+	setup_delays $HEDWIG_NETWORK_DELAY
+    fi
+
+    start_control_server;
+
+    ../test/hedwigtest 
+    RESULT=$?
+    stop_control_server;
+
+    if [ "z$HEDWIG_NETWORK_DELAY" != "z" ]; then
+	clear_delays
+    else
+	cat <<EOF
+
+The environment variable HEDWIG_NETWORK_DELAY is not set, so the tests were run directly 
+with a localhost server. This isn't quite realistic as usually there will be some delay between 
+the client and the hedwig server. Set HEDWIG_NETWORK_DELAY to the number of milliseconds you want
+to delay the packets between the server and client. 
+
+ $ export HEDWIG_NETWORK_DELAY=500
+
+Requires root privileges.
+
+WARNING!!! This will modify your traffic shaping and firewall rules. If you do run with delays, 
+check your firewall rules afterwards.
+
+EOF
+    fi
+
+    exit $RESULT
+}
+
+case "$1" in
+    start-control-server)
+	start_control_server
+	;;
+    stop-control-server)
+	stop_control_server
+	;;
+    setup-delays)
+	setup_delays $2
+	;;
+    clear-delays)
+	clear_delays
+	;;
+    all)
+	all
+	;;
+    *)
+	cat <<EOF
+Usage: tester.sh [command]
+
+tester.sh all
+   Run through the tests, setting up and cleaning up all prerequisites.
+
+tester.sh start-control-server
+   Starts the deamon which the tests use to start and stop hedwig/zookeeper/bookeeper servers
+
+tester.sh stop-control-server
+   Stops the aforementioned daemon
+
+tester.sh setup-delays <delay>
+   Set the millisecond delay for accessing the hedwig servers for the tests.
+
+tester.sh clear-delays
+   Clear the delay for accessing the hedwig servers.
+EOF
+	;;
+esac

+ 23 - 3
src/contrib/hedwig/client/src/main/cpp/test/Makefile.am

@@ -1,6 +1,26 @@
+#
+# 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.
+#
+
 bin_PROGRAMS = hedwigtest
 hedwigtest_SOURCES = main.cpp utiltest.cpp pubsubdatatest.cpp publishtest.cpp subscribetest.cpp servercontrol.cpp pubsubtest.cpp
-hedwigtest_CPPFLAGS = -I../inc $(DEPS_CFLAGS)
-hedwigtest_LDADD = $(DEPS_LIBS) -L../lib -lhedwig01
-hedwigtest_LDFLAGS = -no-undefined
+hedwigtest_CPPFLAGS = -I$(top_srcdir)/inc $(DEPS_CFLAGS) $(BOOST_CPPFLAGS) 
+hedwigtest_LDADD = $(DEPS_LIBS) -L$(top_builddir)/lib -lhedwig01 
+hedwigtest_LDFLAGS = -no-undefined $(BOOST_ASIO_LIB) $(BOOST_LDFLAGS) $(BOOST_THREAD_LIB)
 
+check: hedwigtest
+	bash ../scripts/tester.sh all

+ 16 - 3
src/contrib/hedwig/client/src/main/cpp/test/main.cpp

@@ -23,14 +23,24 @@
 #include <log4cpp/PropertyConfigurator.hh>
 #include <log4cpp/Category.hh>
 #include "servercontrol.h"
+#include "util.h"
 
 #include <cppunit/extensions/TestFactoryRegistry.h>
 #include <cppunit/ui/text/TextTestRunner.h>
 
+#include <cppunit/TextTestProgressListener.h>
+#include <cppunit/TestResult.h>
+
+HedwigCppTextTestProgressListener gprogress;
+
 int main( int argc, char **argv)
 {
   try {
-    log4cpp::PropertyConfigurator::configure("../log4cpp.conf");
+    if (getenv("LOG4CPP_CONF") == NULL) {
+      std::cerr << "Set LOG4CPP_CONF in your environment to get logging." << std::endl;
+    } else {
+      log4cpp::PropertyConfigurator::configure(getenv("LOG4CPP_CONF"));
+    }
   } catch (log4cpp::ConfigureFailure &e) {
     std::cerr << "log4cpp configuration failure while loading : " << e.what() << std::endl;
   } catch (std::exception &e) {
@@ -55,10 +65,13 @@ int main( int argc, char **argv)
     
     runner.addTest( registry.makeTest() );
   }
-  int ret =  runner.run(testPath);
+  
+  runner.eventManager().addListener( &gprogress );
+
+  bool ret = runner.run(testPath);
   google::protobuf::ShutdownProtobufLibrary();
   
   log4cpp::Category::shutdown();
   
-  return ret;
+  return (ret == true) ? 0 : 1;
 }

+ 29 - 11
src/contrib/hedwig/client/src/main/cpp/test/publishtest.cpp

@@ -54,12 +54,10 @@ private:
   CPPUNIT_TEST_SUITE_END();
 
 public:
-  PublishTestSuite() {
-
+  PublishTestSuite() : control(NULL) {
   }
 
   ~PublishTestSuite() {
-
   }
 
   void setUp()
@@ -77,15 +75,29 @@ public:
   
   void tearDown() 
   {
-    hw2->kill();
-    hw1->kill();
+    if (hw2.get()) {
+      hw2->kill();
+    }
+    if (hw1.get()) {
+      hw1->kill();
+    }
     
-    bk1->kill();
-    bk2->kill();
-    bk3->kill();
+    if (bk1.get()) {
+      bk1->kill();
+    }
+    if (bk2.get()) {
+      bk2->kill();
+    }
+    if (bk3.get()) {
+      bk3->kill();
+    }
     
-    zk->kill();
-    delete control;
+    if (zk.get()) {
+      zk->kill();
+    }
+    if (control) {
+      delete control;
+    }
   }
 
   void testSyncPublish() {
@@ -111,6 +123,9 @@ public:
     pub.asyncPublish("testTopic", "async test message", testcb);
     
     cond->wait();
+
+    CPPUNIT_ASSERT(cond->wasSuccess());
+
     delete cond;
     delete client;
     delete conf;
@@ -134,9 +149,12 @@ public:
     pub.asyncPublish("testTopic", "async test message #3", testcb3);
     
     cond3->wait();
+    CPPUNIT_ASSERT(cond3->wasSuccess());
     cond2->wait();
+    CPPUNIT_ASSERT(cond2->wasSuccess());
     cond1->wait();
-
+    CPPUNIT_ASSERT(cond1->wasSuccess());
+    
     delete cond3; delete cond2; delete cond1;
     delete client;
     delete conf;

+ 26 - 15
src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp

@@ -18,6 +18,7 @@
 #include <cppunit/Test.h>
 #include <cppunit/TestSuite.h>
 #include <cppunit/extensions/HelperMacros.h>
+#include <boost/thread/mutex.hpp>
 
 #include "../lib/clientimpl.h"
 #include <hedwig/exceptions.h>
@@ -50,7 +51,7 @@ private:
   CPPUNIT_TEST_SUITE_END();
 
 public:
-  PubSubTestSuite() {
+  PubSubTestSuite() : control(NULL) {
     
   }
 
@@ -72,17 +73,29 @@ public:
   void tearDown() 
   {
     try {
-      hw1->kill();
-    
-      bk1->kill();
-      bk2->kill();
-      bk3->kill();
+      if (hw1.get()) {
+	hw1->kill();
+      }
       
-      zk->kill();
+      if (bk1.get()) {
+	bk1->kill();
+      }
+      if (bk2.get()) {
+	bk2->kill();
+      }
+      if (bk3.get()) {
+	bk3->kill();
+      }
+      
+      if (zk.get()) {
+	zk->kill();
+      }
     } catch (std::exception& e) {
       // don't allow an exception to break everything, we're going deleting the control no matter what
     }
-    delete control;
+    if (control) {
+      delete control;
+    }
   }
 
   class MyMessageHandlerCallback : public Hedwig::MessageHandlerCallback {
@@ -93,30 +106,28 @@ public:
 
     virtual void consume(const std::string& topic, const std::string& subscriberId, const Hedwig::Message& msg, Hedwig::OperationCallbackPtr& callback) {
       if (topic == this->topic && subscriberId == this->subscriberId) {
-	mutex.lock();
+	boost::lock_guard<boost::mutex> lock(mutex);
+      
 	messagesReceived++;
 	lastMessage = msg.body();
 	callback->operationComplete();
-	mutex.unlock();
       }
     }
     
     std::string getLastMessage() {
-      mutex.lock();
+      boost::lock_guard<boost::mutex> lock(mutex);
       std::string s = lastMessage;
-      mutex.unlock();
       return s;
     }
 
     int numMessagesReceived() {
-      mutex.lock();
+      boost::lock_guard<boost::mutex> lock(mutex);
       int i = messagesReceived;
-      mutex.unlock();
       return i;
     }    
     
   protected:
-    Hedwig::Mutex mutex;
+    boost::mutex mutex;
     int messagesReceived;
     std::string lastMessage;
     std::string topic;

+ 9 - 0
src/contrib/hedwig/client/src/main/cpp/test/servercontrol.cpp

@@ -27,11 +27,18 @@
 #include <stdlib.h>
 #include "servercontrol.h"
 
+
 #include <log4cpp/Category.hh>
+
+#include "util.h"
+
 #include <sstream>   
+#include <time.h>
 
 static log4cpp::Category &LOG = log4cpp::Category::getInstance("hedwigtest."__FILE__);
 
+extern HedwigCppTextTestProgressListener gprogress;
+
 using namespace HedwigTest;
 
 const int MAX_COMMAND_LN = 256;
@@ -86,6 +93,8 @@ ServerControl::ServerControl(int port) {
     close(socketfd);
     throw CantConnectToServerControlDaemonException();
   }
+  
+  requestResponse("TEST " + gprogress.getTestName() + "\n");
 }
 
 ServerControl::~ServerControl() {

+ 4 - 2
src/contrib/hedwig/client/src/main/cpp/test/servercontrol.h

@@ -21,10 +21,11 @@
 
 #include <tr1/memory>
 #include <exception>
+#include <boost/thread/mutex.hpp>
 #include "../lib/util.h"
 
 namespace HedwigTest {
-    const int DEFAULT_CONTROLSERVER_PORT = 5672;
+  const int DEFAULT_CONTROLSERVER_PORT = 5672;
 
   class TestException : public std::exception {};
   class CantConnectToServerControlDaemonException : public TestException {};
@@ -36,6 +37,7 @@ namespace HedwigTest {
   public:
     virtual void kill() = 0;
     virtual std::string& getAddress() = 0;
+    virtual ~TestServer() {}
   };
   
   typedef std::tr1::shared_ptr<TestServer> TestServerPtr;
@@ -57,7 +59,7 @@ namespace HedwigTest {
 
   public:
     int socketfd;
-    Hedwig::Mutex socketlock;
+    boost::mutex socketlock;
   };
 };
 

+ 23 - 7
src/contrib/hedwig/client/src/main/cpp/test/subscribetest.cpp

@@ -55,7 +55,7 @@ private:
   CPPUNIT_TEST_SUITE_END();
 
 public:
-  SubscribeTestSuite() {
+  SubscribeTestSuite() : control(NULL) {
     
   }
 
@@ -78,17 +78,29 @@ public:
   void tearDown() 
   {
     try {
-      hw1->kill();
+      if (hw1.get()) {
+	hw1->kill();
+      }
     
-      bk1->kill();
-      bk2->kill();
-      bk3->kill();
+      if (bk1.get()) {
+	bk1->kill();
+      }
+      if (bk2.get()) {
+	bk2->kill();
+      }
+      if (bk3.get()) {
+	bk3->kill();
+      }
       
-      zk->kill();
+      if (zk.get()) {
+	zk->kill();
+      }
     } catch (std::exception& e) {
       // don't allow an exception to break everything, we're going deleting the control no matter what
     }
-    delete control;
+    if (control) {
+      delete control;
+    }
   }
 
   void testSyncSubscribe() {
@@ -132,6 +144,7 @@ public:
     sub.asyncSubscribe("testTopic", "mySubscriberId-3", Hedwig::SubscribeRequest::CREATE_OR_ATTACH, testcb1);
     
     cond1->wait();
+    CPPUNIT_ASSERT(cond1->wasSuccess());
   }
   
   void testAsyncSubcribeAndUnsubscribe() {
@@ -153,9 +166,11 @@ public:
 
     sub.asyncSubscribe("testTopic", "mySubscriberId-4", Hedwig::SubscribeRequest::CREATE_OR_ATTACH, testcb1);
     cond1->wait();
+    CPPUNIT_ASSERT(cond1->wasSuccess());
     
     sub.asyncUnsubscribe("testTopic", "mySubscriberId-4", testcb2);
     cond2->wait();
+    CPPUNIT_ASSERT(cond2->wasSuccess());
   }
 
   void testAsyncSubcribeAndSyncUnsubscribe() {
@@ -174,6 +189,7 @@ public:
     
     sub.asyncSubscribe("testTopic", "mySubscriberId-5", Hedwig::SubscribeRequest::CREATE_OR_ATTACH, testcb1);
     cond1->wait();
+    CPPUNIT_ASSERT(cond1->wasSuccess());
 
     sub.unsubscribe("testTopic", "mySubscriberId-5");
   }

+ 82 - 15
src/contrib/hedwig/client/src/main/cpp/test/util.h

@@ -20,22 +20,50 @@
 #include <hedwig/callback.h>
 #include <stdexcept>
 #include <pthread.h>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/condition_variable.hpp>
 
-static log4cpp::Category &UTILLOG = log4cpp::Category::getInstance("hedwigtest."__FILE__);
 
+#include <cppunit/TextTestProgressListener.h>
+#include <cppunit/TestResult.h>
+#include <cppunit/Test.h>
+
+static log4cpp::Category &UTILLOG = log4cpp::Category::getInstance("hedwigtest."__FILE__);
 
-class SimpleWaitCondition : public Hedwig::WaitConditionBase {
+class SimpleWaitCondition {
 public:
-  SimpleWaitCondition() : flag(false) {};
+ SimpleWaitCondition() : flag(false), success(false) {};
   ~SimpleWaitCondition() { wait(); }
 
-  void setTrue() { UTILLOG.debugStream() << "Setting flag " << &flag << " to true"; flag=true; UTILLOG.debugStream() << "Flag now " << flag; }
-  bool isTrue() {
-    UTILLOG.debugStream() << &flag << " isTrue? " << flag;
-    return flag;
+  void wait() {
+    boost::unique_lock<boost::mutex> lock(mut);
+    while(!flag)
+    {
+        cond.wait(lock);
+    }
+  }
+
+  void notify() {
+    {
+      boost::lock_guard<boost::mutex> lock(mut);
+      flag = true;
+    }
+    cond.notify_all();
+  }
+
+  void setSuccess(bool s) {
+    success = s;
   }
+
+  bool wasSuccess() {
+    return success;
+  }
+
 private:
   bool flag;
+  boost::condition_variable cond;
+  boost::mutex mut;
+  bool success;
 };
 
 class TestCallback : public Hedwig::OperationCallback {
@@ -46,17 +74,18 @@ public:
 
   virtual void operationComplete() {
     UTILLOG.debugStream() << "operationComplete";
-    cond->lock();
-    cond->setTrue();
-    cond->signalAndUnlock();
+    cond->setSuccess(true);
+    cond->notify();
+
   }
   
   virtual void operationFailed(const std::exception& exception) {
     UTILLOG.debugStream() << "operationFailed: " << exception.what();
-    cond->lock();
-    cond->setTrue();
-    cond->signalAndUnlock();
+    cond->setSuccess(false);
+    cond->notify();
   }    
+  
+
 private:
   SimpleWaitCondition *cond;
 };
@@ -66,11 +95,49 @@ class TestServerConfiguration : public Hedwig::Configuration {
 public:
   TestServerConfiguration(HedwigTest::TestServerPtr& server) : server(server), address(server->getAddress()) {}
   
-  virtual const std::string& getDefaultServer() const {
-    return address;
+  virtual int getInt(const std::string& /*key*/, int defaultVal) const {
+    return defaultVal;
+  }
+
+  virtual const std::string get(const std::string& key, const std::string& defaultVal) const {
+    if (key == Configuration::DEFAULT_SERVER) {
+      return address;
+    } else {
+      return defaultVal;
+    }
+  }
+
+  virtual bool getBool(const std::string& /*key*/, bool defaultVal) const {
+    return defaultVal;
   }
   
 private:
   HedwigTest::TestServerPtr server;
   const std::string address;
 };
+
+
+class HedwigCppTextTestProgressListener : public CppUnit::TextTestProgressListener 
+{
+ public:
+  void startTest( CppUnit::Test *test ) {
+    std::cout << "\n****\n\nStarting " << test->getName() << "\n\n****" << std::endl;
+    current_test = test->getName();
+  }
+  
+  void addFailure( const CppUnit::TestFailure &failure ) {
+    std::cout << "\n!!!!!\n\nFailed\n\n!!!!!" << std::endl;
+
+  }
+
+  void endTestRun( CppUnit::Test *test, 
+		   CppUnit::TestResult *eventManager ) {
+  }
+
+  std::string& getTestName() {
+    return current_test;
+  }
+
+private:
+  std::string current_test;
+};

+ 16 - 5
src/contrib/hedwig/server/src/test/java/org/apache/hedwig/ServerControlDaemon.java

@@ -28,7 +28,6 @@ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
 
 import org.apache.log4j.Logger;
 
-
 import org.jboss.netty.channel.Channel;  
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.channel.ChannelEvent;
@@ -74,7 +73,12 @@ public class ServerControlDaemon {
 	    HashMap<String, ServerControl.TestServer> map = serverMap.get(c);
 	    ServerControl.TestServer t = map.get(name);
 	    map.remove(name);
-	    t.kill();
+	    try {
+		t.kill();
+	    } catch (Exception e) {
+		e.printStackTrace();
+		// do nothing, should be killed, we won't use it again anyhow
+	    }
 	}
 
 	private ServerControl.TestServer lookupServer(Channel c, String name) {
@@ -86,10 +90,12 @@ public class ServerControlDaemon {
 	    HashMap<String, ServerControl.TestServer> map = serverMap.get(c);
 	    serverMap.remove(map);
 	    
-	    for (ServerControl.TestServer t : map.values()) {
-		t.kill();
+	    if (map != null) {
+		for (ServerControl.TestServer t : map.values()) {
+		    t.kill();
+		}
+		map.clear();
 	    }
-	    map.clear();
 	}
 
 	public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
@@ -116,11 +122,16 @@ public class ServerControlDaemon {
 		    ctx.getChannel().write("OK " + t.getAddress() + "\n");
 		} else if (args[0].equals("KILL")) {
 		    killServerForChannel(ctx.getChannel(), args[1]);
+		    
 		    ctx.getChannel().write("OK Killed " + args[1] + "\n");
+		} else if (args[0].equals("TEST")) {
+		    LOG.info("\n******\n\n" + args[1] + "\n\n******");
+		    ctx.getChannel().write("OK Test Noted\n");
 		} else {
 		    ctx.getChannel().write("ERR Bad Command\n");
 		}
 	    } catch (Exception ex) {
+		ex.printStackTrace();
 		ctx.getChannel().write("ERR " + ex.toString() + "\n");
 	    }
 	}