You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ac...@apache.org on 2007/11/23 00:55:42 UTC

svn commit: r597513 - in /incubator/qpid/trunk/qpid/cpp: rubygen/templates/ src/ src/qpid/ src/qpid/broker/ src/qpid/client/ src/qpid/cluster/ src/qpid/framing/ src/qpid/management/ src/tests/

Author: aconway
Date: Thu Nov 22 15:55:39 2007
New Revision: 597513

URL: http://svn.apache.org/viewvc?rev=597513&view=rev
Log:

Added framing::BodyHolder:
 - Uniform holder for all body types, replaces MethodHolder.
 - Uses in_place constructors to avoid avoid body copy.

framing::AMQFrame:
 - Holds body in heap-allocated intrusive_ptr<BodyHolder>
 - Uses in_place constructors to avoid avoid body copy.

Removed/downgraded to TODO many redundant FIXME comments.

Added:
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h   (with props)
Removed:
    incubator/qpid/trunk/qpid/cpp/src/qpid/ExceptionHolder.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/ExceptionHolder.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/MethodHolder.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/MethodHolder.h
    incubator/qpid/trunk/qpid/cpp/src/tests/MessageHandlerTest.cpp
Modified:
    incubator/qpid/trunk/qpid/cpp/rubygen/templates/MethodHolder.rb
    incubator/qpid/trunk/qpid/cpp/src/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerAdapter.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/DtxManager.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageAdapter.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageDelivery.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/ExecutionHandler.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/SessionCore.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/SessionManager.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQMethodBody.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameDefaultVisitor.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/variant.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/management/ManagementAgent.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/Blob.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/ClientSessionTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/Cluster.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/Cluster_child.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h
    incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h
    incubator/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/TxAckTest.cpp

Modified: incubator/qpid/trunk/qpid/cpp/rubygen/templates/MethodHolder.rb
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/rubygen/templates/MethodHolder.rb?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/rubygen/templates/MethodHolder.rb (original)
+++ incubator/qpid/trunk/qpid/cpp/rubygen/templates/MethodHolder.rb Thu Nov 22 15:55:39 2007
@@ -7,13 +7,16 @@
   def initialize(outdir, amqp)
     super(outdir, amqp)
     @namespace="qpid::framing"
-    @classname="MethodHolder"
-    @filename="qpid/framing/MethodHolder"
+    @classname="BodyHolder"
+    @filename="qpid/framing/BodyHolder"
   end
 
   def gen_max_size()
     # Generate program to generate MaxSize.h
-    cpp_file("generate_#{@classname}MaxSize_h") {
+    cpp_file("generate_MaxMethodBodySize_h") {
+      include "qpid/framing/AMQHeaderBody"
+      include "qpid/framing/AMQContentBody"
+      include "qpid/framing/AMQHeartbeatBody"
       @amqp.methods_.each { |m| include "qpid/framing/#{m.body_name}" }
       genl
       include "<algorithm>"
@@ -24,13 +27,16 @@
       genl
       scope("int main(int, char** argv) {") {
         genl "size_t maxSize=0;"
+        genl "maxSize=max(maxSize, sizeof(AMQHeaderBody));" 
+        genl "maxSize=max(maxSize, sizeof(AMQContentBody));" 
+        genl "maxSize=max(maxSize, sizeof(AMQHeartbeatBody));" 
         @amqp.methods_.each { |m|
           genl "maxSize=max(maxSize, sizeof(#{m.body_name}));" }
         gen <<EOS
-ofstream out("#{@filename}MaxSize.h");
+ofstream out("qpid/framing/MaxMethodBodySize.h");
 out << "// GENERATED CODE: generated by " << argv[0] << endl;
 out << "namespace qpid{ namespace framing { " << endl;
-out << "const size_t MAX_METHODBODY_SIZE=" << maxSize << ";" << endl;
+out << "const size_t MAX_METHOD_BODY_SIZE=" << maxSize << ";" << endl;
 out << "}}" << endl;
 EOS
       }
@@ -38,40 +44,48 @@
   end
 
   def gen_construct
-    cpp_file(@filename+"_construct") {
+    cpp_file(@filename+"_gen") {
       include @filename
-      include "qpid/framing/MethodBodyConstVisitor.h"
+      include "qpid/framing/AMQHeaderBody"
+      include "qpid/framing/AMQContentBody"
+      include "qpid/framing/AMQHeartbeatBody"
       @amqp.methods_.each { |m| include "qpid/framing/#{m.body_name}" }
-      genl
+      include "qpid/framing/FrameDefaultVisitor.h"
       include "qpid/Exception.h"
       genl
       namespace(@namespace) {
-        # construct function
-        scope("void #{@classname}::construct(ClassId c, MethodId m) {") {
+        scope("void #{@classname}::setMethod(ClassId c, MethodId m) {") {
           scope("switch (c) {") {
             @amqp.classes.each { |c|
               scope("case #{c.index}: switch(m) {") {
                 c.methods_.each { |m|
-                  genl "case #{m.index}: blob.construct(in_place<#{m.body_name}>()); break;"
+                  genl "case #{m.index}: blob = in_place<#{m.body_name}>(); break;"
                 }
-                genl "default: throw Exception(QPID_MSG(\"Invalid method id \" << m << \" for class #{c.name} \"));"
+                genl "default: throw Exception(QPID_MSG(\"Invalid method id \" << int(m) << \" for class #{c.name} \"));"
               }
               genl "break;"
             }
-            genl "default: throw Exception(QPID_MSG(\"Invalid class id \" << c));"
+            genl "default: throw Exception(QPID_MSG(\"Invalid class id \" << int(c)));"
           }
         }
-        # CopyVisitor
-        struct("#{@classname}::CopyVisitor", "public MethodBodyConstVisitor") {           genl "MethodHolder& holder;"
-          genl "CopyVisitor(MethodHolder& h) : holder(h) {}"
+
+        struct("CopyVisitor", "public FrameDefaultVisitor") {
+          genl "using FrameDefaultVisitor::visit;"
+          genl "using FrameDefaultVisitor::defaultVisit;"
+          genl "BodyHolder& holder;"
+          genl "CopyVisitor(BodyHolder& h) : holder(h) {}"
+          ["Header", "Content", "Heartbeat"].each { |type|
+            genl "void visit(const AMQ#{type}Body& x) { holder=x; }"
+          }
           @amqp.methods_.each { |m|
-            genl "void visit(const #{m.body_name}& x) { holder.blob=x; }"
+            genl "void visit(const #{m.body_name}& x) { holder=x; }"
           }
+          genl "void defaultVisit(const AMQBody&) { assert(0); }"
         }
         genl
-        # operator=
-        scope("#{@classname}& MethodHolder::operator=(const AMQMethodBody& m) {") {
-          genl "CopyVisitor cv(*this); m.accept(cv); return *this;"
+
+        scope("void BodyHolder::setBody(const AMQBody& b) {") {
+          genl "CopyVisitor cv(*this); b.accept(cv);"
         }
       }}
   end

Modified: incubator/qpid/trunk/qpid/cpp/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/Makefile.am?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/Makefile.am Thu Nov 22 15:55:39 2007
@@ -27,12 +27,12 @@
 DISTCLEANFILES=rubygen.mk
 
 # Code generated by C++
-noinst_PROGRAMS=generate_MethodHolderMaxSize_h
-generate_MethodHolderMaxSize_h_SOURCES=gen/generate_MethodHolderMaxSize_h.cpp
-qpid/framing/MethodHolderMaxSize.h: generate_MethodHolderMaxSize_h
-	./generate_MethodHolderMaxSize_h
-BUILT_SOURCES=qpid/framing/MethodHolderMaxSize.h
-DISTCLEANFILES+=qpid/framing/MethodHolderMaxSize.h
+noinst_PROGRAMS=generate_MaxMethodBodySize_h
+generate_MaxMethodBodySize_h_SOURCES=gen/generate_MaxMethodBodySize_h.cpp
+qpid/framing/MaxMethodBodySize.h: generate_MaxMethodBodySize_h
+	./generate_MaxMethodBodySize_h
+BUILT_SOURCES=qpid/framing/MaxMethodBodySize.h
+DISTCLEANFILES+=qpid/framing/MaxMethodBodySize.h
 
 
 ## Compiler flags
@@ -99,6 +99,7 @@
   qpid/framing/AMQHeaderBody.cpp \
   qpid/framing/AMQHeartbeatBody.cpp \
   qpid/framing/Array.cpp \
+  qpid/framing/BodyHolder.cpp \
   qpid/framing/BodyHandler.cpp \
   qpid/framing/ChannelAdapter.cpp \
   qpid/framing/Buffer.cpp \
@@ -117,8 +118,7 @@
   qpid/framing/Uuid.cpp \
   qpid/framing/AMQP_HighestVersion.h \
   qpid/framing/Blob.cpp \
-  qpid/framing/MethodHolder.cpp \
-  qpid/framing/MethodHolderMaxSize.h \
+  qpid/framing/MaxMethodBodySize.h \
   qpid/framing/TransferContent.cpp \
   qpid/assert.cpp qpid/assert.h \
   qpid/Exception.cpp \
@@ -226,7 +226,6 @@
   $(platform_hdr) \
   qpid/assert.h \
   qpid/Exception.h \
-  qpid/ExceptionHolder.h \
   qpid/Msg.h \
   qpid/Options.h \
   qpid/Plugin.h \
@@ -359,8 +358,8 @@
   qpid/framing/Invoker.h \
   qpid/framing/InputHandler.h \
   qpid/framing/MethodContent.h \
-  qpid/framing/MethodHolder.h \
-  qpid/framing/MethodHolderMaxSize.h \
+  qpid/framing/BodyHolder.h \
+  qpid/framing/MaxMethodBodySize.h \
   qpid/framing/ModelMethod.h \
   qpid/framing/OutputHandler.h \
   qpid/framing/ProtocolInitiation.h \

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerAdapter.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerAdapter.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerAdapter.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerAdapter.cpp Thu Nov 22 15:55:39 2007
@@ -31,7 +31,7 @@
 
 typedef std::vector<Queue::shared_ptr> QueueVector;
 
-// FIXME aconway 2007-08-31: now that functionality is distributed
+// TODO aconway 2007-08-31: now that functionality is distributed
 // between different handlers, BrokerAdapter should be dropped.
 // Instead the individual class Handler interfaces can be implemented
 // by the handlers responsible for those classes.

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/DtxManager.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/DtxManager.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/DtxManager.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/DtxManager.cpp Thu Nov 22 15:55:39 2007
@@ -31,9 +31,7 @@
 
 DtxManager::DtxManager(TransactionalStore* const _store) : store(_store) {}
 
-DtxManager::~DtxManager() {
-    // timer.stop(); // FIXME aconway 2007-10-23: leaking threads.
-}
+DtxManager::~DtxManager() {}
 
 void DtxManager::start(const std::string& xid, DtxBuffer::shared_ptr ops)
 {

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp Thu Nov 22 15:55:39 2007
@@ -165,7 +165,7 @@
         for (uint64_t offset = 0; offset < expectedSize; offset += maxContentSize)
         {            
             uint64_t remaining = expectedSize - offset;
-            AMQFrame frame(0, AMQContentBody());
+            AMQFrame frame(in_place<AMQContentBody>());
             string& data = frame.castBody<AMQContentBody>()->getData();
 
             store->loadContent(queue, *this, data, offset,
@@ -196,7 +196,7 @@
     frames.map_if(f, TypeFilter<HEADER_BODY>());    
 }
 
-// FIXME aconway 2007-11-09: Obsolete, remove. Was used to cover over
+// TODO aconway 2007-11-09: Obsolete, remove. Was used to cover over
 // 0-8/0-9 message differences.
 MessageAdapter& Message::getAdapter() const
 {

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageAdapter.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageAdapter.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageAdapter.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageAdapter.h Thu Nov 22 15:55:39 2007
@@ -33,7 +33,7 @@
 namespace qpid {	
 namespace broker {
 
-// FIXME aconway 2007-11-09: No longer needed, we only have one type of message.
+// TODO aconway 2007-11-09: No longer needed, we only have one type of message.
 struct MessageAdapter
 {
     virtual ~MessageAdapter() {}

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageDelivery.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageDelivery.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageDelivery.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/MessageDelivery.cpp Thu Nov 22 15:55:39 2007
@@ -52,8 +52,9 @@
 
     AMQFrame sendMethod(intrusive_ptr<Message> msg, DeliveryId id)
     {
-        return AMQFrame(0, BasicGetOkBody(
-            ProtocolVersion(), id.getValue(), msg->getRedelivered(), msg->getExchangeName(),
+        return AMQFrame(in_place<BasicGetOkBody>(
+            ProtocolVersion(), id.getValue(),
+            msg->getRedelivered(), msg->getExchangeName(),
             msg->getRoutingKey(), queue->getMessageCount())); 
     }
 };
@@ -68,9 +69,10 @@
 
     AMQFrame sendMethod(intrusive_ptr<Message> msg, DeliveryId id)
     {
-        return AMQFrame(0, BasicDeliverBody(
+        return AMQFrame(in_place<BasicDeliverBody>(
             ProtocolVersion(), consumer, id.getValue(),
-            msg->getRedelivered(), msg->getExchangeName(), msg->getRoutingKey()));
+            msg->getRedelivered(), msg->getExchangeName(),
+            msg->getRoutingKey()));
     }
 
 };
@@ -90,7 +92,9 @@
         if (msg->getRedelivered()){
             msg->getProperties<DeliveryProperties>()->setRedelivered(true);
         }
-        return AMQFrame(0, MessageTransferBody(ProtocolVersion(), 0, destination, confirmMode, acquireMode));
+        return AMQFrame(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, destination,
+                            confirmMode, acquireMode));
     }
 };
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.h Thu Nov 22 15:55:39 2007
@@ -57,7 +57,7 @@
 
     SemanticState state;
     SessionState& session;
-    // FIXME aconway 2007-09-20: Why are these on the handler rather than the
+    // TODO aconway 2007-09-20: Why are these on the handler rather than the
     // state?
     IncomingExecutionContext incoming;
     framing::Window outgoing;

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp Thu Nov 22 15:55:39 2007
@@ -124,13 +124,13 @@
 
 void  SessionHandler::flow(bool /*active*/) {
     assertAttached("flow");
-    // FIXME aconway 2007-09-19: Removed in 0-10, remove 
+    // TODO aconway 2007-09-19: Removed in 0-10, remove 
     assert(0); throw NotImplementedException("session.flow");
 }
 
 void  SessionHandler::flowOk(bool /*active*/) {
     assertAttached("flowOk");
-    // FIXME aconway 2007-09-19: Removed in 0-10, remove 
+    // TODO aconway 2007-09-19: Removed in 0-10, remove 
     assert(0); throw NotImplementedException("session.flowOk");
 }
 
@@ -181,7 +181,7 @@
 }
 
 void  SessionHandler::highWaterMark(uint32_t /*lastSentMark*/) {
-    // FIXME aconway 2007-10-02: may be removed from spec.
+    // TODO aconway 2007-10-02: may be removed from spec.
     assert(0); throw NotImplementedException("session.high-water-mark");
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp Thu Nov 22 15:55:39 2007
@@ -46,7 +46,7 @@
       version(h.getConnection().getVersion()),
       semanticHandler(new SemanticHandler(*this))
 {
-    // FIXME aconway 2007-09-20: SessionManager may add plugin
+    // TODO aconway 2007-09-20: SessionManager may add plugin
     // handlers to the chain.
  }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.cpp Thu Nov 22 15:55:39 2007
@@ -33,10 +33,6 @@
 #include <boost/bind.hpp>
 #include "qpid/framing/all_method_bodies.h"
 
-// FIXME aconway 2007-01-26: Evaluate all throws, ensure consistent
-// handling of errors that should close the connection or the channel.
-// Make sure the user thread receives a connection in each case.
-//
 using namespace std;
 using namespace boost;
 using namespace qpid::framing;

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/Channel.h Thu Nov 22 15:55:39 2007
@@ -93,7 +93,6 @@
 
     void dispatch(framing::FrameSet& msg, const std::string& destination);
 
-    // FIXME aconway 2007-02-23: Get rid of friendships.
     friend class Connection;
 
   public:

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp Thu Nov 22 15:55:39 2007
@@ -112,7 +112,7 @@
 
 void ConnectionHandler::send(const framing::AMQBody& body)
 {
-    AMQFrame f(0, body);
+    AMQFrame f(body);
     out(f);
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp Thu Nov 22 15:55:39 2007
@@ -97,7 +97,7 @@
 
 void ConnectionImpl::idleOut()
 {
-    AMQFrame frame(0, new AMQHeartbeatBody());
+    AMQFrame frame(in_place<AMQHeartbeatBody>());
     connector->send(frame);
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/ExecutionHandler.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/ExecutionHandler.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/ExecutionHandler.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/ExecutionHandler.cpp Thu Nov 22 15:55:39 2007
@@ -118,7 +118,7 @@
 void ExecutionHandler::sendFlushRequest()
 {
     Mutex::ScopedLock l(lock);
-    AMQFrame frame(0, ExecutionFlushBody());
+    AMQFrame frame(in_place<ExecutionFlushBody>());
     out(frame);
 }
 
@@ -134,7 +134,7 @@
 void ExecutionHandler::sendSyncRequest()
 {
     Mutex::ScopedLock l(lock);
-    AMQFrame frame(0, ExecutionSyncBody());
+    AMQFrame frame(in_place<ExecutionSyncBody>());
     out(frame);
 }
 
@@ -161,7 +161,9 @@
     Mutex::ScopedLock l(lock);
     SequenceNumberSet range;
     incomingCompletionStatus.collectRanges(range);
-    AMQFrame frame(0, ExecutionCompleteBody(version, incomingCompletionStatus.mark.getValue(), range));
+    AMQFrame frame(
+        in_place<ExecutionCompleteBody>(
+            version, incomingCompletionStatus.mark.getValue(), range));
     out(frame);    
 }
 
@@ -177,7 +179,7 @@
     if(l) {
         completion.listenForResult(id, l);
     }
-    AMQFrame frame(0/*channel will be filled in by channel handler*/, command);
+    AMQFrame frame(command);
     if (hasContent) {
         frame.setEof(false);
     }
@@ -196,7 +198,7 @@
 
 void ExecutionHandler::sendContent(const MethodContent& content)
 {
-    AMQFrame header(0, content.getHeader());
+    AMQFrame header(content.getHeader());
     header.setBof(false);
     u_int64_t data_length = content.getData().length();
     if(data_length > 0){
@@ -205,7 +207,7 @@
         //frame itself uses 8 bytes
         u_int32_t frag_size = maxFrameSize - 8;
         if(data_length < frag_size){
-            AMQFrame frame(0, AMQContentBody(content.getData()));
+            AMQFrame frame(in_place<AMQContentBody>(content.getData()));
             frame.setBof(false);
             out(frame);
         }else{
@@ -214,7 +216,7 @@
             while (remaining > 0) {
                 u_int32_t length = remaining > frag_size ? frag_size : remaining;
                 string frag(content.getData().substr(offset, length));
-                AMQFrame frame(0, AMQContentBody(frag));
+                AMQFrame frame(in_place<AMQContentBody>(frag));
                 frame.setBof(false);
                 if (offset > 0) {
                     frame.setBos(false);

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.cpp Thu Nov 22 15:55:39 2007
@@ -32,13 +32,13 @@
 {
     waitForCompletion();
     session.assertOpen();            
-    return response.get();
+    return response.getMethod();
 }
 
 void FutureResponse::received(const AMQMethodBody* r)
 {
     Monitor::ScopedLock l(lock);
-    response = *r;
+    response.setBody(*r);
     complete = true;
     lock.notifyAll();
 }

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/FutureResponse.h Thu Nov 22 15:55:39 2007
@@ -23,7 +23,7 @@
 #define _FutureResponse_
 
 #include "qpid/framing/amqp_framing.h"
-#include "qpid/framing/MethodHolder.h"
+#include "qpid/framing/BodyHolder.h"
 #include "FutureCompletion.h"
 
 namespace qpid {
@@ -33,7 +33,7 @@
 
 class FutureResponse : public FutureCompletion 
 {
-    framing::MethodHolder response;
+    framing::BodyHolder response;
 public:
     framing::AMQMethodBody* getResponse(SessionCore& session);
     void received(const framing::AMQMethodBody* response);

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/SessionCore.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/SessionCore.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/SessionCore.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/SessionCore.cpp Thu Nov 22 15:55:39 2007
@@ -403,7 +403,7 @@
 }
 
 void  SessionCore::highWaterMark(uint32_t /*lastSentMark*/) {
-    // FIXME aconway 2007-10-02: may be removed from spec.
+    // TODO aconway 2007-10-02: may be removed from spec.
     assert(0); throw NotImplementedException("session.highWaterMark");
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp Thu Nov 22 15:55:39 2007
@@ -88,7 +88,7 @@
 }
 
 void Cluster::notify() {
-    AMQFrame frame(0, ClusterNotifyBody(ProtocolVersion(), url));
+    AMQFrame frame(in_place<ClusterNotifyBody>(ProtocolVersion(), url));
     handle(frame);
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/SessionManager.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/SessionManager.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/SessionManager.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/SessionManager.h Thu Nov 22 15:55:39 2007
@@ -57,10 +57,6 @@
     /** ChannelUpdater: add cluster handlers to session. */
     void update(framing::ChannelId, framing::FrameHandler::Chains&);
 
-    // FIXME aconway 2007-08-30: Need setUp and tearDown instead of just
-    // update, so we can tear down closed sesions.
-    // Or add FrameHandler::destroy(Session) to notify all handlers?
-
     /** FrameHandler: map frames from the cluster to sessions. */
     void handle(framing::AMQFrame&);
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h Thu Nov 22 15:55:39 2007
@@ -22,7 +22,6 @@
  *
  */
 #include "qpid/framing/amqp_types.h"
-#include "qpid/shared_ptr.h"
 
 #include <ostream>
 
@@ -47,8 +46,6 @@
 class AMQBody
 {
   public:
-    typedef shared_ptr<AMQBody> shared_ptr;
-
     virtual ~AMQBody();
 
     virtual uint8_t type() const = 0;

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp Thu Nov 22 15:55:39 2007
@@ -31,49 +31,16 @@
 namespace qpid {
 namespace framing {
 
-namespace {
-struct GetBodyVisitor : public NoBlankVisitor<AMQBody*> {
-    QPID_USING_NOBLANK(AMQBody*);
-    AMQBody* operator()(MethodHolder& t) const { return t.get(); }
-    template <class T> AMQBody* operator()(T& t) const { return &t; }
-};
-
-struct EncodeVisitor : public NoBlankVisitor<void> {
-    Buffer& buffer;
-    EncodeVisitor(Buffer& b) : buffer(b) {}
-    
-    QPID_USING_NOBLANK(void);
-    template <class T> void operator()(const T& t) const { return t.encode(buffer); }
-};
-
-struct SizeVisitor : public NoBlankVisitor<uint32_t> {
-    QPID_USING_NOBLANK(uint32_t);
-    template <class T> uint32_t operator()(const T& t) const { return t.size(); }
-};
-
-struct DecodeVisitor : public NoBlankVisitor<void> {
-    Buffer& buffer;
-    uint32_t size;
-    DecodeVisitor(Buffer& b, uint32_t s) : buffer(b), size(s) {}
-    QPID_USING_NOBLANK(void);
-    void operator()(MethodHolder& t) const { return t.decode(buffer); }
-    template <class T> void operator()(T& t) const { return t.decode(buffer, size); }
-};
-
-}
+AMQFrame::~AMQFrame() {}
 
-AMQBody* AMQFrame::getBody() {
-    return boost::apply_visitor(GetBodyVisitor(), body);
-}
+void AMQFrame::setBody(const AMQBody& b) { body = new BodyHolder(b); }
 
-const AMQBody* AMQFrame::getBody() const {
-    return boost::apply_visitor(GetBodyVisitor(), const_cast<Variant&>(body));
-}
+void AMQFrame::setMethod(ClassId c, MethodId m) { body = new BodyHolder(c,m); }
 
-// This is now misleadingly named as it is not the frame size as defined in the spec 
-// (as it also includes the end marker)
-uint32_t AMQFrame::size() const{
-    return frameOverhead() + boost::apply_visitor(SizeVisitor(), body);
+// This is now misleadingly named as it is not the frame size as
+// defined in the spec (as it also includes the end marker)
+uint32_t AMQFrame::size() const {
+    return frameOverhead() + body->size();
 }
 
 uint32_t AMQFrame::frameOverhead() {
@@ -90,7 +57,7 @@
     buffer.putOctet(0x0f & subchannel);
     buffer.putShort(channel);    
     buffer.putLong(0);
-    boost::apply_visitor(EncodeVisitor(buffer), body);
+    body->encode(buffer);
     buffer.putOctet(0xCE);
 }
 
@@ -119,45 +86,34 @@
     (void) buffer.getLong(); // reserved2
     
     // Verify that the protocol header meets current spec
-    // TODO: should we check reserved2 against zero as well? - the spec isn't clear
+    // TODO: should we check reserved2 against zero as well? - the
+    // spec isn't clear
     if ((flags & 0x30) != 0 || reserved1 != 0 || (field1 & 0xf0) != 0)
         throw SyntaxErrorException(QPID_MSG("Reserved bits not zero"));
 
-    // TODO: should no longer care about body size and only pass up B,E,b,e flags
+    // TODO: should no longer care about body size and only pass up
+    // B,E,b,e flags
     uint16_t body_size = frame_size + 1 - frameOverhead(); 
     if (buffer.available() < body_size+1u){
         buffer.restore();
         return false;
     }
-    decodeBody(buffer, body_size, type);
-
+    body = new BodyHolder();
+    body->decode(type,buffer, body_size);
     uint8_t end = buffer.getOctet();
     if (end != 0xCE)
         throw SyntaxErrorException(QPID_MSG("Frame end not found"));
     return true;
 }
 
-void AMQFrame::decodeBody(Buffer& buffer, uint32_t size, uint8_t type)
-{    
-    switch(type)
-    {
-      case METHOD_BODY: body = MethodHolder(); break;
-      case HEADER_BODY: body = AMQHeaderBody();	break;
-      case CONTENT_BODY: body = AMQContentBody(); break;
-      case HEARTBEAT_BODY: body = AMQHeartbeatBody(); break;
-
-      default:
-	throw SyntaxErrorException(QPID_MSG("Invalid frame type " << type));
-    }
-    boost::apply_visitor(DecodeVisitor(buffer,size), body);
-}
-
 std::ostream& operator<<(std::ostream& out, const AMQFrame& f)
 {
-    return out << "Frame[" 
-               << (f.getBof() ? "B" : "") << (f.getEof() ? "E" : "") << (f.getBos() ? "b" : "") << (f.getEos() ? "e" : "") << "; "
-               << "channel=" << f.getChannel() << "; " << *f.getBody()
-               << "]";
+    return
+        out << "Frame[" 
+            << (f.getBof() ? "B" : "") << (f.getEof() ? "E" : "")
+            << (f.getBos() ? "b" : "") << (f.getEos() ? "e" : "") << "; "
+            << "channel=" << f.getChannel() << "; " << *f.getBody()
+            << "]";
 }
 
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h Thu Nov 22 15:55:39 2007
@@ -25,46 +25,48 @@
 #include "AMQHeaderBody.h"
 #include "AMQContentBody.h"
 #include "AMQHeartbeatBody.h"
-#include "MethodHolder.h"
 #include "ProtocolVersion.h"
+#include "BodyHolder.h"
 
 #include <boost/cast.hpp>
-#include <boost/variant.hpp>
 
 namespace qpid {
 namespace framing {
-	
+
+class BodyHolder;
+
 class AMQFrame : public AMQDataBlock
 {
   public:
-    AMQFrame() : bof(true), eof(true), bos(true), eos(true), subchannel(0), channel(0) {}
-
-    /** Construct a frame with a copy of b */
-    AMQFrame(ChannelId c, const AMQBody* b) : bof(true), eof(true), bos(true), eos(true), subchannel(0), channel(c) {
-        setBody(*b);
-    }
-    
-    AMQFrame(ChannelId c, const AMQBody& b) : bof(true), eof(true), bos(true), eos(true), subchannel(0), channel(c) {
-        setBody(b);
-    }
-    
-    AMQFrame(const AMQBody& b) : bof(true), eof(true), bos(true), eos(true), subchannel(0), channel(0) {
-        setBody(b);
+    AMQFrame(intrusive_ptr<BodyHolder> b=0) : body(b) { init(); }
+    AMQFrame(const AMQBody& b) { setBody(b); init(); }
+    ~AMQFrame();
+
+    template <class InPlace>
+    AMQFrame(const InPlace& ip, typename EnableInPlace<InPlace>::type* =0) {
+        init(); setBody(ip);
     }
-    
+
     ChannelId getChannel() const { return channel; }
     void setChannel(ChannelId c) { channel = c; }
 
-    AMQBody* getBody();
-    const AMQBody* getBody() const;
+    intrusive_ptr<BodyHolder> getHolder() { return body; }
+    
+    AMQBody* getBody() { return body ? body->get() : 0; }
+    const AMQBody* getBody() const { return body ? body->get() : 0; }
 
     AMQMethodBody* getMethod() { return getBody()->getMethod(); }
     const AMQMethodBody* getMethod() const { return getBody()->getMethod(); }
 
-    /** Copy a body instance to the frame */
-    void setBody(const AMQBody& b) { CopyVisitor cv(*this); b.accept(cv); }
+    void setBody(const AMQBody& b);
+
+    template <class InPlace>
+    typename EnableInPlace<InPlace>::type setBody(const InPlace& ip) {
+        body = new BodyHolder(ip);
+    }
+
+    void setMethod(ClassId c, MethodId m);
 
-    /** Convenience template to cast the body to an expected type. */
     template <class T> T* castBody() {
         return boost::polymorphic_downcast<T*>(getBody());
     }
@@ -73,8 +75,6 @@
         return boost::polymorphic_downcast<const T*>(getBody());
     }
 
-    bool empty() { return boost::get<boost::blank>(&body); }
-
     void encode(Buffer& buffer) const; 
     bool decode(Buffer& buffer); 
     uint32_t size() const;
@@ -92,33 +92,15 @@
     static uint32_t frameOverhead();
 
   private:
-    struct CopyVisitor : public AMQBodyConstVisitor {
-        AMQFrame& frame;
-        CopyVisitor(AMQFrame& f) : frame(f) {}
-        void visit(const AMQHeaderBody& x) { frame.body=x; }
-        void visit(const AMQContentBody& x) { frame.body=x; }
-        void visit(const AMQHeartbeatBody& x) { frame.body=x; }
-        void visit(const AMQMethodBody& x) { frame.body=MethodHolder(x); }
-    };
-    friend struct CopyVisitor;
-
-    typedef boost::variant<boost::blank,
-                           AMQHeaderBody,
-                           AMQContentBody,
-                           AMQHeartbeatBody,
-                           MethodHolder> Variant;
-
-    void visit(AMQHeaderBody& x) { body=x; }
-
-    void decodeBody(Buffer& buffer, uint32_t size, uint8_t type);
-
-    bool bof;
-    bool eof;
-    bool bos;
-    bool eos;
-    uint8_t subchannel;
-    uint16_t channel;
-    Variant body;
+    void init() { bof = eof = bos = eos = true; subchannel=0; channel=0; }
+
+    intrusive_ptr<BodyHolder> body;
+    uint16_t channel : 16;
+    uint8_t subchannel : 8;
+    bool bof : 1;
+    bool eof : 1;
+    bool bos : 1;
+    bool eos : 1;
 };
 
 std::ostream& operator<<(std::ostream&, const AMQFrame&);

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQMethodBody.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQMethodBody.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQMethodBody.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQMethodBody.h Thu Nov 22 15:55:39 2007
@@ -40,8 +40,6 @@
 class AMQMethodBody : public AMQBody {
   public:
     AMQMethodBody() {}
-    AMQMethodBody(uint8_t, uint8_t) {}
-    
     virtual ~AMQMethodBody();
 
     virtual void accept(MethodBodyConstVisitor&) const = 0;

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h Thu Nov 22 15:55:39 2007
@@ -25,38 +25,45 @@
 #include <boost/aligned_storage.hpp>
 #include <boost/checked_delete.hpp>
 #include <boost/utility/typed_in_place_factory.hpp>
+#include <boost/type_traits/is_base_and_derived.hpp>
+#include <boost/utility/enable_if.hpp>
 
 #include <new>
 
 #include <assert.h>
 
 
-namespace boost {
+namespace qpid {
+namespace framing {
 
-/**
- * 0-arg typed_in_place_factory constructor and in_place() override.
- * 
- * Boost doesn't provide the 0 arg version since it assumes
- * in_place_factory will be used when there is no default ctor.
- */
+using boost::in_place;          
+using boost::typed_in_place_factory_base;
+
+/** 0-arg typed_in_place_factory, missing in boost. */
 template <class T>
-class typed_in_place_factory0 : public typed_in_place_factory_base {
-  public:
+struct typed_in_place_factory0 : public typed_in_place_factory_base {
     typedef T value_type ; 
     void apply ( void* address ) const { new (address) T(); }
 };
 
+/** 0-arg in_place<T>() function, missing from boost. */
 template<class T>
 typed_in_place_factory0<T> in_place() { return typed_in_place_factory0<T>(); }
 
-} // namespace boost
-
-
-namespace qpid {
-namespace framing {
-
-using boost::in_place;
-
+template <class T, class R=void>
+struct EnableInPlace
+    : public boost::enable_if<boost::is_base_and_derived<
+                                  typed_in_place_factory_base, T>,
+                              R>
+{};
+       
+template <class T, class R=void>
+struct DisableInPlace
+    : public boost::disable_if<boost::is_base_and_derived<
+                                   typed_in_place_factory_base, T>,
+                               R>
+{};
+       
 template <class T> struct BlobHelper {
     static void destroy(void* ptr) { static_cast<T*>(ptr)->~T(); }
     static void copy(void* dest, const void* src) {
@@ -108,11 +115,10 @@
         basePtr=0;
     }
 
-    template<class TypedInPlaceFactory>
-    void construct (const TypedInPlaceFactory& factory,
-                    const boost::typed_in_place_factory_base* )
+    template<class Factory>
+    typename EnableInPlace<Factory>::type apply(const Factory& factory)
     {
-        typedef typename TypedInPlaceFactory::value_type T;
+        typedef typename Factory::value_type T;
         assert(empty());
         factory.apply(store.address());
         setType<T>();
@@ -135,28 +141,31 @@
     Blob(const Blob& b) { initialize(); assign(b); }
 
     /** @see construct() */
-    template<class Expr>
-    Blob( const Expr & expr ) { initialize(); construct(expr,&expr); }
+    template<class InPlace>
+    Blob(const InPlace & expr, typename EnableInPlace<InPlace>::type* =0) {
+        initialize(); apply(expr);
+    }
 
     ~Blob() { clear(); }
 
-    /** Assign a blob */
+    /** Assign from another blob. */
     Blob& operator=(const Blob& b) {
         clear();
         assign(b);
         return *this;
     }
 
-    /** Construcct an object in the blob. Destroyes the previous object.
-     *@param expr an expresion of the form: in_place<T>(x,y,z)
-     * will construct an object using the constructor T(x,y,z)
-     */
-    template<class Expr> void
-    construct(const Expr& expr) { clear(); construct(expr,&expr); }
+    /** Assign from an in_place constructor expression. */
+    template<class InPlace>
+    typename EnableInPlace<InPlace,Blob&>::type operator=(const InPlace& expr) {
+        clear(); apply(expr); return *this;
+    }
 
-    /** Copy construct an instance of T into the Blob. */
+    /** Assign from an object of type T. */
     template <class T>
-    Blob& operator=(const T& x) { clear(); construct(in_place<T>(x)); return *this; }
+    typename DisableInPlace<T, Blob&>::type operator=(const T& x) {
+        clear(); apply(in_place<T>(x)); return *this;
+    }
 
     /** Get pointer to blob contents, returns 0 if empty. */
     BaseType* get() { return  basePtr; }

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp?rev=597513&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp Thu Nov 22 15:55:39 2007
@@ -0,0 +1,75 @@
+/*
+ *
+ * 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 "BodyHolder.h"
+#include "AMQMethodBody.h"
+#include "AMQHeaderBody.h"
+#include "AMQContentBody.h"
+#include "AMQHeartbeatBody.h"
+#include "Buffer.h"
+#include "qpid/framing/reply_exceptions.h"
+
+namespace qpid {
+namespace framing {
+
+
+// BodyHolder::operator=(const AMQBody&)  is defined
+// in generated file BodyHolder_gen.cpp
+
+
+void BodyHolder::encode(Buffer& b) const {
+    const AMQMethodBody* method=getMethod();
+    if (method) {
+        b.putOctet(method->amqpClassId());
+        b.putOctet(method->amqpMethodId());
+        method->encode(b);
+    }
+    else
+        get()->encode(b);
+}
+
+void BodyHolder::decode(uint8_t type, Buffer& buffer, uint32_t size) {
+    switch(type)
+    {
+      case METHOD_BODY: {
+          ClassId c = buffer.getOctet();
+          MethodId m = buffer.getOctet();
+          setMethod(c, m);
+          break;
+      }
+      case HEADER_BODY: *this=in_place<AMQHeaderBody>(); break;
+      case CONTENT_BODY: *this=in_place<AMQContentBody>(); break;
+      case HEARTBEAT_BODY: *this=in_place<AMQHeartbeatBody>(); break;
+      default:
+	throw SyntaxErrorException(QPID_MSG("Invalid frame type " << type));
+    }
+    get()->decode(buffer, size);
+}
+
+uint32_t BodyHolder::size() const {
+    const AMQMethodBody* method=getMethod();
+    if (method) 
+        return sizeof(ClassId)+sizeof(MethodId)+method->size();
+    else
+        return get()->size();
+}
+
+}} // namespace qpid::framing
+

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h?rev=597513&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h Thu Nov 22 15:55:39 2007
@@ -0,0 +1,88 @@
+#ifndef QPID_FRAMING_BODYHOLDER_H
+#define QPID_FRAMING_BODYHOLDER_H
+
+/*
+ *
+ * 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 "qpid/framing/AMQBody.h"
+#include "qpid/framing/Blob.h"
+#include "qpid/framing/MaxMethodBodySize.h" // Generated file.
+#include "qpid/framing/amqp_types.h"
+#include "qpid/RefCounted.h"
+
+
+namespace qpid {
+namespace framing {
+
+class AMQMethodBody;
+class AMQBody;
+class Buffer;
+
+/**
+ * Holder for arbitrary frame body.
+ */
+class BodyHolder : public RefCounted
+{
+  public:
+    // default copy, assign dtor ok.
+    BodyHolder() {}
+    BodyHolder(const AMQBody& b) { setBody(b); }
+    BodyHolder(ClassId c, MethodId m) { setMethod(c,m); }
+
+    /** Construct from an in_place constructor expression */
+    template <class InPlace>
+    BodyHolder(const InPlace& ip, typename EnableInPlace<InPlace>::type* =0)
+        : blob(ip) {}
+
+    void setBody(const AMQBody& b);
+
+    /** Assign from an in_place constructor expression */
+    template <class InPlace>
+    typename EnableInPlace<InPlace,BodyHolder&>::type
+    operator=(const InPlace& ip) { blob=ip; return *this; }
+
+    /** Assign by copying. */
+    template <class T>
+    typename DisableInPlace<T,BodyHolder&>::type operator=(const T& x)
+    { blob=in_place<T>(x); return *this; }
+
+    /** Set to method with ClassId c, MethodId m. */
+    void setMethod(ClassId c, MethodId m);
+
+    void encode(Buffer&) const;
+    void decode(uint8_t frameType, Buffer&, uint32_t=0);
+    uint32_t size() const;
+
+    /** Return body pointer or 0 if empty. */
+    AMQBody* get() { return blob.get(); }
+    const AMQBody* get() const { return blob.get(); }
+
+    /** Return method pointer or 0 if not a method. */
+    AMQMethodBody* getMethod() { return get()->getMethod(); }
+    const AMQMethodBody* getMethod() const { return get()->getMethod(); }
+
+  private:
+    Blob<MAX_METHOD_BODY_SIZE, AMQBody> blob;
+};
+
+}} // namespace qpid::framing
+
+#endif  /*!QPID_FRAMING_BODYHOLDER_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp Thu Nov 22 15:55:39 2007
@@ -43,7 +43,8 @@
 void ChannelAdapter::send(const AMQBody& body)
 {
     assertChannelOpen();
-    AMQFrame frame(getId(), body);
+    AMQFrame frame(body);
+    frame.setChannel(getId());
     handlers.out(frame);
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameDefaultVisitor.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameDefaultVisitor.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameDefaultVisitor.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameDefaultVisitor.h Thu Nov 22 15:55:39 2007
@@ -24,8 +24,6 @@
 #include "qpid/framing/MethodBodyDefaultVisitor.h"
 #include "qpid/framing/AMQBody.h"
 #include "qpid/framing/AMQMethodBody.h"
-#include "qpid/framing/AMQFrame.h"
-#include "qpid/framing/FrameHandler.h"
 
 namespace qpid {
 namespace framing {

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h Thu Nov 22 15:55:39 2007
@@ -27,7 +27,7 @@
 namespace qpid {
 namespace framing {
 
-// FIXME aconway 2007-08-29: Eliminate, replace with FrameHandler.
+// TODO aconway 2007-08-29: Eliminate, replace with FrameHandler.
 class InputHandler : public FrameHandler {
   public:
     virtual ~InputHandler() {}

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h Thu Nov 22 15:55:39 2007
@@ -27,7 +27,7 @@
 namespace qpid {
 namespace framing {
 
-// FIXME aconway 2007-08-29: Replace with FrameHandler.
+// TODO aconway 2007-08-29: Replace with FrameHandler.
 class OutputHandler : public FrameHandler {
   public:
     virtual ~OutputHandler() {}

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp Thu Nov 22 15:55:39 2007
@@ -51,7 +51,8 @@
 
 void qpid::framing::SendContent::sendFragment(const AMQContentBody& body, uint32_t offset, uint16_t size, bool first, bool last) const
 {
-    AMQFrame fragment(0, AMQContentBody(body.getData().substr(offset, size)));
+    AMQFrame fragment(in_place<AMQContentBody>(
+                          body.getData().substr(offset, size)));
     setFlags(fragment, first, last);
     handler.handle(fragment);
 }

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/variant.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/variant.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/variant.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/variant.h Thu Nov 22 15:55:39 2007
@@ -32,7 +32,7 @@
 
 /** boost::static_visitor that throws exception if variant contains blank.
  * Sublclasses need to have a using() declaration, can be generated
- * with QPID_USING_BLANK_THROW(R)
+ * with QPID_USING_NOBLANK(R)
  */
 template <class R=void>
 struct NoBlankVisitor : public boost::static_visitor<R> {

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/management/ManagementAgent.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/management/ManagementAgent.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/management/ManagementAgent.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/management/ManagementAgent.cpp Thu Nov 22 15:55:39 2007
@@ -98,15 +98,14 @@
                                   string   routingKey)
 {
     intrusive_ptr<Message> msg (new Message ());
-    AMQFrame method  (0, MessageTransferBody(ProtocolVersion(),
-                                             0, exchange->getName (), 0, 0));
-    AMQFrame header  (0, AMQHeaderBody());
-    AMQFrame content;
+    AMQFrame method (in_place<MessageTransferBody>(
+                         ProtocolVersion(), 0, exchange->getName (), 0, 0));
+    AMQFrame header (in_place<AMQHeaderBody>());
+    AMQFrame content(in_place<AMQContentBody>());
 
     QPID_LOG (debug, "ManagementAgent::SendBuffer - key="
               << routingKey << " len=" << length);
 
-    content.setBody(AMQContentBody());
     content.castBody<AMQContentBody>()->decode(buf, length);
 
     method.setEof  (false);

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/Blob.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/Blob.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/Blob.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/Blob.cpp Thu Nov 22 15:55:39 2007
@@ -95,7 +95,7 @@
         BOOST_CHECK_EQUAL(1, Foo::instances);
         BOOST_CHECK_EQUAL(1, Bar::instances);
 
-        b2.construct(in_place<Foo>(4)); 
+        b2 = in_place<Foo>(4); 
         BOOST_CHECK_EQUAL(4, b2.get()->id);
         BOOST_CHECK_EQUAL(2, Foo::instances);
         BOOST_CHECK_EQUAL(0, Bar::instances);

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp Thu Nov 22 15:55:39 2007
@@ -346,8 +346,9 @@
     {
         intrusive_ptr<Message> msg(new Message());
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
 
         msg->getFrames().append(method);
         msg->getFrames().append(header);
@@ -360,7 +361,7 @@
 
     void addContent(intrusive_ptr<Message> msg, const string& data)
     {
-        AMQFrame content(0, AMQContentBody(data));
+        AMQFrame content(in_place<AMQContentBody>(data));
         msg->getFrames().append(content);
     }
 };

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/ClientSessionTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/ClientSessionTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/ClientSessionTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/ClientSessionTest.cpp Thu Nov 22 15:55:39 2007
@@ -71,7 +71,6 @@
     CPPUNIT_TEST(testUseSuspendedError);
     CPPUNIT_TEST(testSuspendResume);
     CPPUNIT_TEST(testDisconnectResume);
-    CPPUNIT_TEST(testAutoDelete);
     CPPUNIT_TEST_SUITE_END();
 
     shared_ptr<broker::Broker> broker;
@@ -97,9 +96,7 @@
     void declareSubscribe(const std::string& q="my-queue",
                           const std::string& dest="my-dest")
     {
-        // FIXME aconway 2007-10-18: autoDelete queues are destroyed on channel close, not session.
-        // Fix & make all test queues exclusive, autoDelete
-        session.queueDeclare(queue=q);  // FIXME aconway 2007-10-01: exclusive=true, autoDelete=true);
+        session.queueDeclare(queue=q);
         session.messageSubscribe(queue=q, destination=dest, acquireMode=1);
         session.messageFlow(destination=dest, unit=0, value=0xFFFFFFFF);//messages
         session.messageFlow(destination=dest, unit=1, value=0xFFFFFFFF);//bytes
@@ -202,27 +199,6 @@
         c->disconnect(); // Simulate disconnect, resume on a new connection.
         c2->resume(session);
         CPPUNIT_ASSERT(queueExists("after"));
-    }
-
-    void testAutoDelete() {
-        // Verify that autoDelete queues survive suspend/resume.
-        session = c->newSession(60);
-        session.queueDeclare(queue="my-queue", exclusive=true, autoDelete=true);
-        CPPUNIT_ASSERT(queueExists("my-queue"));
-        session.suspend();
-        c->resume(session);
-        CPPUNIT_ASSERT(queueExists("my-queue"));
-
-        // Verify they survive disconnect/resume on new Connection
-        c->disconnect();
-        c2->resume(session);
-
-        try { 
-            // FIXME aconway 2007-10-23: Negative test, need to
-            // fix auto-delete queues to clean up with session, not channel.
-            CPPUNIT_ASSERT(queueExists("my-queue"));
-            CPPUNIT_FAIL("Negative test passed unexpectedly");
-        } catch(const ChannelException&) {}
     }
 };
 

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/Cluster.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/Cluster.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/Cluster.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/Cluster.cpp Thu Nov 22 15:55:39 2007
@@ -34,7 +34,8 @@
 /** Verify membership in a cluster with one member. */
 BOOST_AUTO_TEST_CASE(testClusterOne) {
     TestCluster cluster("clusterOne", "amqp:one:1");
-    AMQFrame send(1, SessionOpenBody(VER));
+    AMQFrame send(in_place<SessionOpenBody>(VER));
+    send.setChannel(1);
     cluster.handle(send);
     AMQFrame received = cluster.received.pop();
     BOOST_CHECK_TYPEID_EQUAL(SessionOpenBody, *received.getBody());
@@ -59,7 +60,8 @@
         BOOST_REQUIRE(cluster.waitFor(2)); // Myself and child.
 
         // Exchange frames with child.
-        AMQFrame send(1, SessionOpenBody(VER));
+        AMQFrame send(SessionOpenBody(VER));
+        send.setChannel(1);
         cluster.handle(send);
         AMQFrame received = cluster.received.pop();
         BOOST_CHECK_TYPEID_EQUAL(SessionOpenBody, *received.getBody());
@@ -89,8 +91,8 @@
     
 /** Test the ClassifierHandler */
 BOOST_AUTO_TEST_CASE(testClassifierHandlerWiring) {
-    AMQFrame queueDecl(0, QueueDeclareBody(VER));
-    AMQFrame messageTrans(0, MessageTransferBody(VER));
+    AMQFrame queueDecl(in_place<QueueDeclareBody>(VER));
+    AMQFrame messageTrans(in_place<MessageTransferBody>(VER));
     CountHandler wiring;
     CountHandler other;
     

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/Cluster_child.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/Cluster_child.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/Cluster_child.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/Cluster_child.cpp Thu Nov 22 15:55:39 2007
@@ -39,7 +39,8 @@
     BOOST_CHECK_TYPEID_EQUAL(SessionOpenBody, *frame.getBody());
     BOOST_CHECK_EQUAL(2u, cluster.size()); // Me and parent
 
-    AMQFrame send(1, SessionAttachedBody(VER));
+    AMQFrame send(in_place<SessionAttachedBody>(VER));
+    send.setChannel(1);
     cluster.handle(send);
     BOOST_REQUIRE(cluster.received.waitPop(frame));
     BOOST_CHECK_TYPEID_EQUAL(SessionAttachedBody, *frame.getBody());

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp Thu Nov 22 15:55:39 2007
@@ -147,7 +147,8 @@
         Buffer wbuff(buffer, sizeof(buffer));
         std::string a = "hostA";
         std::string b = "hostB";
-        AMQFrame in(999, ConnectionRedirectBody(version, a, b));
+        AMQFrame in(in_place<ConnectionRedirectBody>(version, a, b));
+        in.setChannel(999);
         in.encode(wbuff);
 
         Buffer rbuff(buffer, sizeof(buffer));
@@ -160,7 +161,8 @@
     {
         Buffer wbuff(buffer, sizeof(buffer));
         std::string s = "hostA";
-        AMQFrame in(999, BasicConsumeOkBody(version, s));
+        AMQFrame in(in_place<BasicConsumeOkBody>(version, s));
+        in.setChannel(999);
         in.encode(wbuff);
 
         Buffer rbuff(buffer, sizeof(buffer));

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp Thu Nov 22 15:55:39 2007
@@ -56,7 +56,7 @@
     }
 
     void testMessageProperties() {
-        AMQFrame out(0, AMQHeaderBody());
+        AMQFrame out(in_place<AMQHeaderBody>());
         MessageProperties* props1 = 
             out.castBody<AMQHeaderBody>()->get<MessageProperties>(true);
 
@@ -96,7 +96,7 @@
     }
 
     void testDeliveryProperies() {
-        AMQFrame out(0, AMQHeaderBody());
+        AMQFrame out(in_place<AMQHeaderBody>());
         DeliveryProperties* props1 = 
             out.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true);
 

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am Thu Nov 22 15:55:39 2007
@@ -66,7 +66,6 @@
   TxAckTest		\
   TxBufferTest		\
   TxPublishTest		\
-  MessageHandlerTest    \
   MessageBuilderTest    \
   ClientSessionTest
 

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp Thu Nov 22 15:55:39 2007
@@ -100,8 +100,9 @@
         std::string exchange("builder-exchange");
         std::string key("builder-exchange");
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
 
         header.castBody<AMQHeaderBody>()->get<MessageProperties>(true)->setContentLength(0);        
         header.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true)->setRoutingKey(key);
@@ -123,9 +124,9 @@
         std::string exchange("builder-exchange");
         std::string key("builder-exchange");
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
-        AMQFrame content(0, AMQContentBody(data));
+        AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
+        AMQFrame content(in_place<AMQContentBody>(data));
         method.setEof(false);
         header.setBof(false);
         header.setEof(false);
@@ -156,10 +157,11 @@
         std::string exchange("builder-exchange");
         std::string key("builder-exchange");
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
-        AMQFrame content1(0, AMQContentBody(data1));
-        AMQFrame content2(0, AMQContentBody(data2));
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
+        AMQFrame content1(in_place<AMQContentBody>(data1));
+        AMQFrame content2(in_place<AMQContentBody>(data2));
         method.setEof(false);
         header.setBof(false);
         header.setEof(false);
@@ -191,10 +193,11 @@
         std::string exchange("builder-exchange");
         std::string key("builder-exchange");
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
-        AMQFrame content1(0, AMQContentBody(data1));
-        AMQFrame content2(0, AMQContentBody(data2));
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
+        AMQFrame content1(in_place<AMQContentBody>(data1));
+        AMQFrame content2(in_place<AMQContentBody>(data2));
 
         header.castBody<AMQHeaderBody>()->get<MessageProperties>(true)->setContentLength(data1.size() + data2.size());        
         header.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true)->setRoutingKey(key);

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp Thu Nov 22 15:55:39 2007
@@ -51,10 +51,11 @@
 
         intrusive_ptr<Message> msg(new Message());
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
-        AMQFrame content1(0, AMQContentBody(data1));
-        AMQFrame content2(0, AMQContentBody(data2));
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
+        AMQFrame content1(in_place<AMQContentBody>(data1));
+        AMQFrame content2(in_place<AMQContentBody>(data2));
 
         msg->getFrames().append(method);
         msg->getFrames().append(header);

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h Thu Nov 22 15:55:39 2007
@@ -34,8 +34,8 @@
     {
         intrusive_ptr<Message> msg(new Message());
 
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
+        AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
 
         msg->getFrames().append(method);
         msg->getFrames().append(header);
@@ -48,7 +48,7 @@
 
     static void addContent(intrusive_ptr<Message> msg, const string& data)
     {
-        AMQFrame content(0, AMQContentBody(data));
+        AMQFrame content(in_place<AMQContentBody>(data));
         msg->getFrames().append(content);
     }
 };

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h Thu Nov 22 15:55:39 2007
@@ -37,12 +37,6 @@
  */
 struct MockChannel : public qpid::framing::ChannelAdapter
 {
-    typedef qpid::framing::BasicGetBody Body;
-    static Body::shared_ptr basicGetBody() {
-        return Body::shared_ptr(
-            new Body(qpid::framing::ProtocolVersion()));
-    }
-
     MockOutputHandler out;
 
     MockChannel(qpid::framing::ChannelId id) {

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp Thu Nov 22 15:55:39 2007
@@ -73,8 +73,9 @@
   public:
     intrusive_ptr<Message> message(std::string exchange, std::string routingKey) {
         intrusive_ptr<Message> msg(new Message());
-        AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, exchange, 0, 0));
-        AMQFrame header(0, AMQHeaderBody());
+        AMQFrame method(in_place<MessageTransferBody>(
+                            ProtocolVersion(), 0, exchange, 0, 0));
+        AMQFrame header(in_place<AMQHeaderBody>());
         msg->getFrames().append(method);
         msg->getFrames().append(header);
         msg->getFrames().getHeaders()->get<DeliveryProperties>(true)->setRoutingKey(routingKey);

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/TxAckTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/TxAckTest.cpp?rev=597513&r1=597512&r2=597513&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/TxAckTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/TxAckTest.cpp Thu Nov 22 15:55:39 2007
@@ -70,8 +70,9 @@
     {
         for(int i = 0; i < 10; i++){
             intrusive_ptr<Message> msg(new Message());
-            AMQFrame method(0, MessageTransferBody(ProtocolVersion(), 0, "exchange", 0, 0));
-            AMQFrame header(0, AMQHeaderBody());
+            AMQFrame method(in_place<MessageTransferBody>(
+                                ProtocolVersion(), 0, "exchange", 0, 0));
+            AMQFrame header(in_place<AMQHeaderBody>());
             msg->getFrames().append(method);
             msg->getFrames().append(header);
             msg->getProperties<DeliveryProperties>()->setDeliveryMode(PERSISTENT);