summaryrefslogtreecommitdiff
path: root/cpp/src
diff options
context:
space:
mode:
authorGordon Sim <gsim@apache.org>2007-09-12 14:49:12 +0000
committerGordon Sim <gsim@apache.org>2007-09-12 14:49:12 +0000
commit46b15e2c2f9e454fd7041f1e6ad723d093bee052 (patch)
tree34af4f367c3b5b46955ce070384a99adb742b562 /cpp/src
parent86255826a849d7d8ced73009274a5bee909a0923 (diff)
downloadqpid-python-46b15e2c2f9e454fd7041f1e6ad723d093bee052.tar.gz
In ClientChannel: Use subscribe and flush in place of get; use per-subscriber flow control for managing prefetches.
In brokers Session: set credit to 0 when subscription is created (modified python tests accordingly) git-svn-id: https://svn.apache.org/repos/asf/incubator/qpid/trunk/qpid@574979 13f79535-47bb-0310-9956-ffa450edef68
Diffstat (limited to 'cpp/src')
-rw-r--r--cpp/src/Makefile.am3
-rw-r--r--cpp/src/qpid/broker/BrokerQueue.cpp2
-rw-r--r--cpp/src/qpid/broker/Session.cpp4
-rw-r--r--cpp/src/qpid/client/BlockingQueue.h8
-rw-r--r--cpp/src/qpid/client/ChannelHandler.cpp10
-rw-r--r--cpp/src/qpid/client/ChannelHandler.h5
-rw-r--r--cpp/src/qpid/client/ClientChannel.cpp64
-rw-r--r--cpp/src/qpid/client/ClientChannel.h9
-rw-r--r--cpp/src/qpid/client/Demux.cpp119
-rw-r--r--cpp/src/qpid/client/Demux.h96
-rw-r--r--cpp/src/qpid/client/Execution.h2
-rw-r--r--cpp/src/qpid/client/ExecutionHandler.cpp2
-rw-r--r--cpp/src/qpid/client/ExecutionHandler.h6
-rw-r--r--cpp/src/qpid/client/MessageQueue.h52
-rw-r--r--cpp/src/qpid/client/SessionCore.cpp7
-rw-r--r--cpp/src/tests/ClientSessionTest.cpp2
16 files changed, 346 insertions, 45 deletions
diff --git a/cpp/src/Makefile.am b/cpp/src/Makefile.am
index a1d8e38372..217d3c3b4c 100644
--- a/cpp/src/Makefile.am
+++ b/cpp/src/Makefile.am
@@ -212,6 +212,7 @@ libqpidclient_la_SOURCES = \
qpid/client/ClientQueue.cpp \
qpid/client/ConnectionImpl.cpp \
qpid/client/Connector.cpp \
+ qpid/client/Demux.cpp \
qpid/client/MessageListener.cpp \
qpid/client/Correlator.cpp \
qpid/client/CompletionTracker.cpp \
@@ -297,7 +298,9 @@ nobase_include_HEADERS = \
qpid/client/ConnectionImpl.h \
qpid/client/Connector.h \
qpid/client/Completion.h \
+ qpid/client/Demux.h \
qpid/client/MessageListener.h \
+ qpid/client/MessageQueue.h \
qpid/client/BlockingQueue.h \
qpid/client/Correlator.h \
qpid/client/CompletionTracker.h \
diff --git a/cpp/src/qpid/broker/BrokerQueue.cpp b/cpp/src/qpid/broker/BrokerQueue.cpp
index 1a13a31a5e..d96622cd4f 100644
--- a/cpp/src/qpid/broker/BrokerQueue.cpp
+++ b/cpp/src/qpid/broker/BrokerQueue.cpp
@@ -126,7 +126,7 @@ bool Queue::acquire(const QueuedMessage& msg) {
void Queue::requestDispatch(Consumer* c, bool sync){
if (!c || c->preAcquires()) {
if (sync) {
- serializer.dispatch();
+ dispatch();
} else {
serializer.execute(dispatchCallback);
}
diff --git a/cpp/src/qpid/broker/Session.cpp b/cpp/src/qpid/broker/Session.cpp
index 650182c807..c98fdd6291 100644
--- a/cpp/src/qpid/broker/Session.cpp
+++ b/cpp/src/qpid/broker/Session.cpp
@@ -251,8 +251,8 @@ Session::ConsumerImpl::ConsumerImpl(Session* _parent,
acquire(_acquire),
blocked(false),
windowing(true),
- msgCredit(0xFFFFFFFF),
- byteCredit(0xFFFFFFFF) {}
+ msgCredit(0),
+ byteCredit(0) {}
bool Session::ConsumerImpl::deliver(QueuedMessage& msg)
{
diff --git a/cpp/src/qpid/client/BlockingQueue.h b/cpp/src/qpid/client/BlockingQueue.h
index 7081b76b68..a9d8ec2857 100644
--- a/cpp/src/qpid/client/BlockingQueue.h
+++ b/cpp/src/qpid/client/BlockingQueue.h
@@ -62,7 +62,7 @@ public:
}
}
- void push(T t)
+ void push(const T& t)
{
sys::Monitor::ScopedLock l(lock);
bool wasEmpty = queue.empty();
@@ -78,6 +78,12 @@ public:
closed = true;
lock.notifyAll();
}
+
+ bool empty()
+ {
+ sys::Monitor::ScopedLock l(lock);
+ return queue.empty();
+ }
};
}}
diff --git a/cpp/src/qpid/client/ChannelHandler.cpp b/cpp/src/qpid/client/ChannelHandler.cpp
index 754b0544c6..c9b7a68f38 100644
--- a/cpp/src/qpid/client/ChannelHandler.cpp
+++ b/cpp/src/qpid/client/ChannelHandler.cpp
@@ -36,7 +36,9 @@ void ChannelHandler::incoming(AMQFrame& frame)
ChannelCloseBody* closeBody=
dynamic_cast<ChannelCloseBody*>(body->getMethod());
if (closeBody) {
- setState(CLOSED);
+ setState(CLOSED_BY_PEER);
+ code = closeBody->getReplyCode();
+ text = closeBody->getReplyText();
if (onClose) {
onClose(closeBody->getReplyCode(), closeBody->getReplyText());
}
@@ -65,8 +67,10 @@ void ChannelHandler::outgoing(AMQFrame& frame)
if (getState() == OPEN) {
frame.setChannel(id);
out(frame);
- } else {
+ } else if (getState() == CLOSED) {
throw Exception("Channel not open");
+ } else if (getState() == CLOSED_BY_PEER) {
+ throw ChannelException(code, text);
}
}
@@ -80,7 +84,7 @@ void ChannelHandler::open(uint16_t _id)
std::set<int> states;
states.insert(OPEN);
- states.insert(CLOSED);
+ states.insert(CLOSED_BY_PEER);
waitFor(states);
if (getState() != OPEN) {
throw Exception("Failed to open channel.");
diff --git a/cpp/src/qpid/client/ChannelHandler.h b/cpp/src/qpid/client/ChannelHandler.h
index 556e13a4f8..24c24e49c4 100644
--- a/cpp/src/qpid/client/ChannelHandler.h
+++ b/cpp/src/qpid/client/ChannelHandler.h
@@ -30,9 +30,12 @@ namespace client {
class ChannelHandler : private StateManager, public ChainableFrameHandler
{
- enum STATES {OPENING, OPEN, CLOSING, CLOSED};
+ enum STATES {OPENING, OPEN, CLOSING, CLOSED, CLOSED_BY_PEER};
framing::ProtocolVersion version;
uint16_t id;
+
+ uint16_t code;
+ std::string text;
void handleMethod(framing::AMQMethodBody* method);
diff --git a/cpp/src/qpid/client/ClientChannel.cpp b/cpp/src/qpid/client/ClientChannel.cpp
index a014fd90c5..87062e1470 100644
--- a/cpp/src/qpid/client/ClientChannel.cpp
+++ b/cpp/src/qpid/client/ClientChannel.cpp
@@ -26,8 +26,10 @@
#include "ClientMessage.h"
#include "qpid/QpidError.h"
#include "Connection.h"
+#include "Demux.h"
#include "FutureResponse.h"
#include "MessageListener.h"
+#include "MessageQueue.h"
#include <boost/format.hpp>
#include <boost/bind.hpp>
#include "qpid/framing/all_method_bodies.h"
@@ -72,6 +74,9 @@ void Channel::open(const Session& s)
THROW_QPID_ERROR(INTERNAL_ERROR, "Attempt to re-open channel");
active = true;
session = s;
+ if(isTransactional()) {
+ session.txSelect();
+ }
}
bool Channel::isOpen() const {
@@ -79,17 +84,8 @@ bool Channel::isOpen() const {
return active;
}
-void Channel::setQos() {
- session.basicQos((prefetchCount=getPrefetch(), global=false));
- if(isTransactional()) {
- //I think this is wrong! should only send TxSelect once...
- session.txSelect();
- }
-}
-
-void Channel::setPrefetch(uint16_t _prefetch){
+void Channel::setPrefetch(uint32_t _prefetch){
prefetch = _prefetch;
- setQos();
}
void Channel::declareExchange(Exchange& _exchange, bool synch){
@@ -157,6 +153,9 @@ void Channel::consume(
session.messageSubscribe(0, _queue.getName(), tag, noLocal,
confirmMode, 0/*pre-acquire*/,
false, fields ? *fields : FieldTable());
+ //allocate some credit:
+ session.messageFlow(tag, 1/*BYTES*/, 0xFFFFFFFF);
+ session.messageFlow(tag, 0/*MESSAGES*/, prefetch ? prefetch : 0xFFFFFFFF);
}
void Channel::cancel(const std::string& tag, bool synch) {
@@ -173,21 +172,29 @@ void Channel::cancel(const std::string& tag, bool synch) {
session.messageCancel(tag);
}
-bool Channel::get(Message& msg, const Queue& queue, AckMode ackMode) {
- Response response = session.basicGet(0, queue.getName(), ackMode == NO_ACK);
- session.execution().sendFlushRequest();
- if (response.isA<BasicGetEmptyBody>()) {
+bool Channel::get(Message& msg, const Queue& _queue, AckMode ackMode) {
+ string tag = "get-handler";
+ ScopedDivert handler(tag, session.execution().getDemux());
+ Demux::Queue& incoming = handler.getQueue();
+
+ session.messageSubscribe((destination=tag, queue=_queue.getName(), confirmMode=(ackMode == NO_ACK ? 0 : 1)));
+ session.messageFlow(tag, 1/*BYTES*/, 0xFFFFFFFF);
+ session.messageFlow(tag, 0/*MESSAGES*/, 1);
+ Completion status = session.messageFlush(tag);
+ status.sync();
+ session.messageCancel(tag);
+
+ if (incoming.empty()) {
return false;
} else {
- FrameSet::shared_ptr content = gets.pop();
- msg.populate(*content);
+ msg.populate(*(incoming.pop()));
return true;
}
}
void Channel::publish(Message& msg, const Exchange& exchange,
const std::string& routingKey,
- bool mandatory, bool /*immediate TODO-restore immediate?*/) {
+ bool mandatory, bool /*?TODO-restore immediate?*/) {
msg.getDeliveryProperties().setRoutingKey(routingKey);
msg.getDeliveryProperties().setDiscardUnroutable(!mandatory);
@@ -224,14 +231,23 @@ void Channel::join() {
void Channel::dispatch(FrameSet& content, const std::string& destination)
{
- ConsumerMap::iterator i = consumers.find(destination);
- if (i != consumers.end()) {
+ MessageListener* listener(0);
+ {
+ Mutex::ScopedLock l(lock);
+ ConsumerMap::iterator i = consumers.find(destination);
+ if (i != consumers.end()) {
+ Message msg;
+ msg.populate(content);
+ listener = i->second.listener;
+ }
+ }
+ if (listener) {
Message msg;
msg.populate(content);
- i->second.listener->received(msg);
+ listener->received(msg);
} else {
QPID_LOG(warning, "Dropping message for unrecognised consumer: " << destination);
- }
+ }
}
void Channel::run() {
@@ -239,12 +255,8 @@ void Channel::run() {
while (true) {
FrameSet::shared_ptr content = session.get();
//need to dispatch this to the relevant listener:
- if (content->isA<BasicDeliverBody>()) {
- dispatch(*content, content->as<BasicDeliverBody>()->getConsumerTag());
- } else if (content->isA<MessageTransferBody>()) {
+ if (content->isA<MessageTransferBody>()) {
dispatch(*content, content->as<MessageTransferBody>()->getDestination());
- } else if (content->isA<BasicGetOkBody>()) {
- gets.push(content);
} else {
QPID_LOG(warning, "Dropping unsupported message type: " << content->getMethod());
}
diff --git a/cpp/src/qpid/client/ClientChannel.h b/cpp/src/qpid/client/ClientChannel.h
index 9e5e3a2e70..b33af65d21 100644
--- a/cpp/src/qpid/client/ClientChannel.h
+++ b/cpp/src/qpid/client/ClientChannel.h
@@ -71,7 +71,7 @@ class Channel : private sys::Runnable
mutable sys::Mutex lock;
sys::Thread dispatcher;
- uint16_t prefetch;
+ uint32_t prefetch;
const bool transactional;
framing::ProtocolVersion version;
@@ -88,7 +88,6 @@ class Channel : private sys::Runnable
void stop();
- void setQos();
void open(const Session& session);
void closeInternal();
void join();
@@ -110,7 +109,7 @@ class Channel : private sys::Runnable
* messages the channel is willing to have sent to it
* asynchronously
*/
- Channel(bool transactional = false, u_int16_t prefetch = 500);
+ Channel(bool transactional = false, u_int16_t prefetch = 0);
~Channel();
@@ -204,9 +203,9 @@ class Channel : private sys::Runnable
/**
* Change the prefetch in use.
*/
- void setPrefetch(uint16_t prefetch);
+ void setPrefetch(uint32_t prefetch);
- uint16_t getPrefetch() { return prefetch; }
+ uint32_t getPrefetch() { return prefetch; }
/**
* Start message dispatching on a new thread
diff --git a/cpp/src/qpid/client/Demux.cpp b/cpp/src/qpid/client/Demux.cpp
new file mode 100644
index 0000000000..d85ad92003
--- /dev/null
+++ b/cpp/src/qpid/client/Demux.cpp
@@ -0,0 +1,119 @@
+/*
+ *
+ * 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 "Demux.h"
+#include "qpid/Exception.h"
+#include "qpid/framing/MessageTransferBody.h"
+
+#include <iostream>
+
+namespace qpid {
+namespace client {
+
+ByTransferDest::ByTransferDest(const std::string& d) : dest(d) {}
+bool ByTransferDest::operator()(const framing::FrameSet& frameset) const
+{
+ return frameset.isA<framing::MessageTransferBody>() &&
+ frameset.as<framing::MessageTransferBody>()->getDestination() == dest;
+}
+
+ScopedDivert::ScopedDivert(const std::string& _dest, Demux& _demuxer) : dest(_dest), demuxer(_demuxer)
+{
+ queue = &(demuxer.add(dest, ByTransferDest(dest)));
+}
+
+ScopedDivert::~ScopedDivert()
+{
+ demuxer.remove(dest);
+}
+
+Demux::Queue& ScopedDivert::getQueue()
+{
+ return *queue;
+}
+
+void Demux::handle(framing::FrameSet::shared_ptr frameset)
+{
+ sys::Mutex::ScopedLock l(lock);
+ bool matched = false;
+ for (iterator i = records.begin(); i != records.end() && !matched; i++) {
+ if (i->condition && i->condition(*frameset)) {
+ matched = true;
+ i->queue->push(frameset);
+ }
+ }
+ if (!matched) {
+ defaultQueue.push(frameset);
+ }
+}
+
+void Demux::close()
+{
+ sys::Mutex::ScopedLock l(lock);
+ for (iterator i = records.begin(); i != records.end(); i++) {
+ i->queue->close();
+ }
+ defaultQueue.close();
+}
+
+Demux::Queue& Demux::add(const std::string& name, Condition condition)
+{
+ sys::Mutex::ScopedLock l(lock);
+ iterator i = std::find_if(records.begin(), records.end(), Find(name));
+ if (i == records.end()) {
+ Record r(name, condition);
+ records.push_back(r);
+ return *(r.queue);
+ } else {
+ throw Exception("Queue already exists for " + name);
+ }
+}
+
+void Demux::remove(const std::string& name)
+{
+ sys::Mutex::ScopedLock l(lock);
+ records.remove_if(Find(name));
+}
+
+Demux::Queue& Demux::get(const std::string& name)
+{
+ sys::Mutex::ScopedLock l(lock);
+ iterator i = std::find_if(records.begin(), records.end(), Find(name));
+ if (i == records.end()) {
+ throw Exception("No queue for " + name);
+ }
+ return *(i->queue);
+}
+
+Demux::Queue& Demux::getDefault()
+{
+ return defaultQueue;
+}
+
+Demux::Find::Find(const std::string& n) : name(n) {}
+
+bool Demux::Find::operator()(const Record& record) const
+{
+ return record.name == name;
+}
+
+}}
+
diff --git a/cpp/src/qpid/client/Demux.h b/cpp/src/qpid/client/Demux.h
new file mode 100644
index 0000000000..0f261f70ba
--- /dev/null
+++ b/cpp/src/qpid/client/Demux.h
@@ -0,0 +1,96 @@
+/*
+ *
+ * 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 <list>
+#include <boost/function.hpp>
+#include <boost/shared_ptr.hpp>
+#include "qpid/framing/FrameSet.h"
+#include "qpid/sys/Mutex.h"
+#include "BlockingQueue.h"
+
+#ifndef _Demux_
+#define _Demux_
+
+namespace qpid {
+namespace client {
+
+class ByTransferDest
+{
+ const std::string dest;
+public:
+ ByTransferDest(const std::string& dest);
+ bool operator()(const framing::FrameSet& frameset) const;
+};
+
+class Demux
+{
+public:
+ typedef boost::function<bool(const framing::FrameSet&)> Condition;
+ typedef BlockingQueue<framing::FrameSet::shared_ptr> Queue;
+
+ void handle(framing::FrameSet::shared_ptr);
+ void close();
+
+ Queue& add(const std::string& name, Condition);
+ void remove(const std::string& name);
+ Queue& get(const std::string& name);
+ Queue& getDefault();
+private:
+ typedef boost::shared_ptr<Queue> QueuePtr;
+ struct Record
+ {
+ const std::string name;
+ Condition condition;
+ QueuePtr queue;
+
+ Record(const std::string& n, Condition c) : name(n), condition(c), queue(new Queue()) {}
+ };
+
+ sys::Mutex lock;
+ std::list<Record> records;
+ Queue defaultQueue;
+
+ typedef std::list<Record>::iterator iterator;
+
+ struct Find
+ {
+ const std::string name;
+ Find(const std::string& name);
+ bool operator()(const Record& record) const;
+ };
+};
+
+class ScopedDivert
+{
+ const std::string dest;
+ Demux& demuxer;
+ Demux::Queue* queue;
+public:
+ ScopedDivert(const std::string& dest, Demux& demuxer);
+ ~ScopedDivert();
+ Demux::Queue& getQueue();
+};
+
+}
+}
+
+
+#endif
diff --git a/cpp/src/qpid/client/Execution.h b/cpp/src/qpid/client/Execution.h
index 1e8c48734d..809dcc7592 100644
--- a/cpp/src/qpid/client/Execution.h
+++ b/cpp/src/qpid/client/Execution.h
@@ -22,6 +22,7 @@
#define _Execution_
#include "qpid/framing/SequenceNumber.h"
+#include "Demux.h"
namespace qpid {
namespace client {
@@ -33,6 +34,7 @@ public:
virtual void sendSyncRequest() = 0;
virtual void sendFlushRequest() = 0;
virtual void completed(const framing::SequenceNumber& id, bool cumulative, bool send) = 0;
+ virtual Demux& getDemux() = 0;
};
}}
diff --git a/cpp/src/qpid/client/ExecutionHandler.cpp b/cpp/src/qpid/client/ExecutionHandler.cpp
index c2b5e45928..8ea2cc64e6 100644
--- a/cpp/src/qpid/client/ExecutionHandler.cpp
+++ b/cpp/src/qpid/client/ExecutionHandler.cpp
@@ -68,7 +68,7 @@ void ExecutionHandler::handle(AMQFrame& frame)
}
arriving->append(frame);
if (arriving->isComplete()) {
- received.push(arriving);
+ demux.handle(arriving);
arriving.reset();
}
} else {
diff --git a/cpp/src/qpid/client/ExecutionHandler.h b/cpp/src/qpid/client/ExecutionHandler.h
index 3078f6bc3a..88424b555a 100644
--- a/cpp/src/qpid/client/ExecutionHandler.h
+++ b/cpp/src/qpid/client/ExecutionHandler.h
@@ -27,10 +27,10 @@
#include "qpid/framing/FrameSet.h"
#include "qpid/framing/MethodContent.h"
#include "qpid/framing/SequenceNumber.h"
-#include "BlockingQueue.h"
#include "ChainableFrameHandler.h"
#include "CompletionTracker.h"
#include "Correlator.h"
+#include "Demux.h"
#include "Execution.h"
namespace qpid {
@@ -46,7 +46,7 @@ class ExecutionHandler :
framing::FrameSet::shared_ptr arriving;
Correlator correlation;
CompletionTracker completion;
- BlockingQueue<framing::FrameSet::shared_ptr> received;
+ Demux demux;
framing::ProtocolVersion version;
uint64_t maxFrameSize;
framing::AccumulatedAck completionStatus;
@@ -79,7 +79,7 @@ public:
void setMaxFrameSize(uint64_t size) { maxFrameSize = size; }
Correlator& getCorrelator() { return correlation; }
CompletionTracker& getCompletionTracker() { return completion; }
- BlockingQueue<framing::FrameSet::shared_ptr>& getReceived() { return received; }
+ Demux& getDemux() { return demux; }
};
}}
diff --git a/cpp/src/qpid/client/MessageQueue.h b/cpp/src/qpid/client/MessageQueue.h
new file mode 100644
index 0000000000..1f5c492910
--- /dev/null
+++ b/cpp/src/qpid/client/MessageQueue.h
@@ -0,0 +1,52 @@
+/*
+ *
+ * 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 _MessageQueue_
+#define _MessageQueue_
+#include <iostream>
+#include "BlockingQueue.h"
+#include "MessageListener.h"
+
+namespace qpid {
+namespace client {
+
+ /**
+ * A MessageListener implementation that simply queues up
+ * messages.
+ *
+ * \ingroup clientapi
+ */
+ class MessageQueue : public MessageListener, public BlockingQueue<Message>
+ {
+ std::queue<Message> messages;
+ public:
+ void received(Message& msg)
+ {
+ std::cout << "Adding message to queue: " << msg.getData() << std::endl;
+ push(msg);
+ }
+ };
+
+}
+}
+
+
+#endif
diff --git a/cpp/src/qpid/client/SessionCore.cpp b/cpp/src/qpid/client/SessionCore.cpp
index 8dfe42989b..3595479642 100644
--- a/cpp/src/qpid/client/SessionCore.cpp
+++ b/cpp/src/qpid/client/SessionCore.cpp
@@ -50,7 +50,7 @@ ExecutionHandler& SessionCore::getExecution()
FrameSet::shared_ptr SessionCore::get()
{
- return l3.getReceived().pop();
+ return l3.getDemux().getDefault().pop();
}
void SessionCore::setSync(bool s)
@@ -71,7 +71,7 @@ void SessionCore::close()
void SessionCore::stop()
{
- l3.getReceived().close();
+ l3.getDemux().close();
l3.getCompletionTracker().close();
}
@@ -98,6 +98,8 @@ void SessionCore::checkClosed()
Future SessionCore::send(const AMQBody& command)
{
+ checkClosed();
+
Future f;
//any result/response listeners must be set before the command is sent
if (command.getMethod()->resultExpected()) {
@@ -120,6 +122,7 @@ Future SessionCore::send(const AMQBody& command)
Future SessionCore::send(const AMQBody& command, const MethodContent& content)
{
+ checkClosed();
//content bearing methods don't currently have responses or
//results, if that changes should follow procedure for the other
//send method impl:
diff --git a/cpp/src/tests/ClientSessionTest.cpp b/cpp/src/tests/ClientSessionTest.cpp
index 9f68716104..7c58708974 100644
--- a/cpp/src/tests/ClientSessionTest.cpp
+++ b/cpp/src/tests/ClientSessionTest.cpp
@@ -65,6 +65,8 @@ class ClientSessionTest : public CppUnit::TestCase
session.queueDeclare_(queue=queueName, exclusive=true, autoDelete=true);
//subcribe to the queue with confirm_mode = 1:
session.messageSubscribe_(queue=queueName, destination=dest, acquireMode=1);
+ session.messageFlow((destination=dest, unit=0, value=1));//messages
+ session.messageFlow((destination=dest, unit=1, value=0xFFFFFFFF));//bytes
//publish a message:
TransferContent _content(data);
_content.getDeliveryProperties().setRoutingKey("my-queue");