|
@@ -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;
|
|
|
}
|