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 2009/01/22 21:29:14 UTC

svn commit: r736783 - in /qpid/trunk/qpid/cpp: rubygen/framing.0-10/ src/ src/qpid/ src/qpid/broker/ src/qpid/client/ src/qpid/cluster/ src/qpid/framing/ src/qpid/management/ src/qpid/replication/ src/qpid/sys/ src/tests/

Author: aconway
Date: Thu Jan 22 12:29:12 2009
New Revision: 736783

URL: http://svn.apache.org/viewvc?rev=736783&view=rev
Log:
Removed BodyHolder: minor performance improvement, opens the way for more efficient memory management.

Added:
    qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb   (with props)
    qpid/trunk/qpid/cpp/src/qpid/framing/BodyFactory.h   (with props)
    qpid/trunk/qpid/cpp/src/qpid/framing/MethodBodyFactory.h   (with props)
Modified:
    qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodHolder.rb
    qpid/trunk/qpid/cpp/rubygen/framing.0-10/structs.rb
    qpid/trunk/qpid/cpp/src/Makefile.am
    qpid/trunk/qpid/cpp/src/qpid/RefCounted.h
    qpid/trunk/qpid/cpp/src/qpid/broker/DeliveryRecord.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/MessageBuilder.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/SessionImpl.cpp
    qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQContentBody.h
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeaderBody.h
    qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeartbeatBody.h
    qpid/trunk/qpid/cpp/src/qpid/framing/Array.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h
    qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h
    qpid/trunk/qpid/cpp/src/qpid/framing/FieldTable.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/SequenceNumberSet.cpp
    qpid/trunk/qpid/cpp/src/qpid/framing/SequenceSet.cpp
    qpid/trunk/qpid/cpp/src/qpid/management/ManagementBroker.cpp
    qpid/trunk/qpid/cpp/src/qpid/replication/ReplicatingEventListener.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/Thread.h
    qpid/trunk/qpid/cpp/src/tests/Blob.cpp
    qpid/trunk/qpid/cpp/src/tests/ExchangeTest.cpp
    qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp
    qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp
    qpid/trunk/qpid/cpp/src/tests/Makefile.am
    qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp
    qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp
    qpid/trunk/qpid/cpp/src/tests/MessageUtils.h
    qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp
    qpid/trunk/qpid/cpp/src/tests/SessionState.cpp
    qpid/trunk/qpid/cpp/src/tests/ssl_test

Added: qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb?rev=736783&view=auto
==============================================================================
--- qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb (added)
+++ qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb Thu Jan 22 12:29:12 2009
@@ -0,0 +1,58 @@
+#!/usr/bin/env ruby
+#
+# 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 .. in load path
+require 'cppgen'
+
+class MethodBodyFactoryGen < CppGen
+  
+  def initialize(outdir, amqp)
+    super(outdir, amqp)
+    @namespace="qpid::framing"
+    @classname="MethodBodyFactory"
+    @filename="qpid/framing/MethodBodyFactory"
+  end
+
+  def generate()
+    cpp_file(@filename) {
+      include @filename
+      include "qpid/framing/BodyFactory"
+      @amqp.methods_.each { |m| include "qpid/framing/#{m.body_name}" }
+      include "qpid/Exception.h"
+      genl
+      namespace(@namespace) {
+        scope("boost::intrusive_ptr<AMQMethodBody> #{@classname}::create(ClassId c, MethodId m) {") {
+          scope("switch (c) {") {
+            @amqp.classes.each { |c|
+              scope("case #{c.code}: switch(m) {") {
+                c.methods_.each { |m|
+                  genl "case #{m.code}: return BodyFactory::create<#{m.body_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 \" << int(c)));"
+          }
+        }
+      }}
+  end
+end
+
+MethodBodyFactoryGen.new($outdir, $amqp).generate();

Propchange: qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodBodyFactory.rb
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodHolder.rb
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodHolder.rb?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodHolder.rb (original)
+++ qpid/trunk/qpid/cpp/rubygen/framing.0-10/MethodHolder.rb Thu Jan 22 12:29:12 2009
@@ -1,118 +0,0 @@
-#!/usr/bin/env ruby
-#
-# 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 .. in load path
-require 'cppgen'
-
-class MethodHolderGen < CppGen
-  
-  def initialize(outdir, amqp)
-    super(outdir, amqp)
-    @namespace="qpid::framing"
-    @classname="BodyHolder"
-    @filename="qpid/framing/BodyHolder"
-  end
-
-  def gen_max_size()
-    # Generate program to generate 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>"
-      include "<fstream>"
-      genl
-      genl "using namespace std;"
-      genl "using namespace qpid::framing;"
-      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("qpid/framing/MaxMethodBodySize.h");
-out << "// GENERATED CODE: generated by " << argv[0] << endl;
-out << "namespace qpid{ namespace framing { " << endl;
-out << "const size_t MAX_METHOD_BODY_SIZE=" << maxSize << ";" << endl;
-out << "}}" << endl;
-EOS
-      }
-    }
-  end
-
-  def gen_construct
-    cpp_file(@filename+"_gen") {
-      include @filename
-      include "qpid/framing/AMQHeaderBody"
-      include "qpid/framing/AMQContentBody"
-      include "qpid/framing/AMQHeartbeatBody"
-      @amqp.methods_.each { |m| include "qpid/framing/#{m.body_name}" }
-      include "qpid/framing/FrameDefaultVisitor.h"
-      include "qpid/Exception.h"
-      genl
-      namespace(@namespace) {
-        scope("void #{@classname}::setMethod(ClassId c, MethodId m) {") {
-          scope("switch (c) {") {
-            @amqp.classes.each { |c|
-              scope("case #{c.code}: switch(m) {") {
-                c.methods_.each { |m|
-                  genl "case #{m.code}: blob = in_place<#{m.body_name}>(); break;"
-                }
-                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 \" << int(c)));"
-          }
-        }
-
-        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=x; }"
-          }
-          genl "void defaultVisit(const AMQBody&) { assert(0); }"
-        }
-        genl
-
-        scope("void BodyHolder::setBody(const AMQBody& b) {") {
-          genl "CopyVisitor cv(*this); b.accept(cv);"
-        }
-      }}
-  end
-
-  def generate
-    gen_max_size
-    gen_construct
-  end
-end
-
-MethodHolderGen.new($outdir, $amqp).generate();
-

Modified: qpid/trunk/qpid/cpp/rubygen/framing.0-10/structs.rb
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/rubygen/framing.0-10/structs.rb?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/rubygen/framing.0-10/structs.rb (original)
+++ qpid/trunk/qpid/cpp/rubygen/framing.0-10/structs.rb Thu Jan 22 12:29:12 2009
@@ -233,6 +233,7 @@
 
     using  AMQMethodBody::accept;
     void accept(MethodBodyConstVisitor& v) const { v.visit(*this); }
+    boost::intrusive_ptr<AMQBody> clone() const { return BodyFactory::copy(*this); }
 
     ClassId amqpClassId() const { return CLASS_ID; }
     MethodId amqpMethodId() const { return METHOD_ID; }

Modified: qpid/trunk/qpid/cpp/src/Makefile.am
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/Makefile.am?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ qpid/trunk/qpid/cpp/src/Makefile.am Thu Jan 22 12:29:12 2009
@@ -27,7 +27,6 @@
   broker.vcproj \
   common.vcproj \
   client.vcproj \
-  MaxMethodBodySize.vcproj \
   qmfconsole.vcproj \
   protocol_gen.mak \
   qpid/log/windows/SinkOptions.cpp \
@@ -99,14 +98,6 @@
 include $(srcdir)/rubygen.mk
 include $(srcdir)/managementgen.mk
 
-# Code generated by C++
-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
 AM_CXXFLAGS = $(WARNING_CFLAGS)
 AM_LDFLAGS = -version-info $(LIBTOOL_VERSION_INFO_ARG)
@@ -231,7 +222,7 @@
   rdmaconnector.la
 
 # RDMA test/sample programs
-noinst_PROGRAMS += RdmaServer RdmaClient
+noinst_PROGRAMS = RdmaServer RdmaClient
 RdmaServer_SOURCES = qpid/sys/rdma/RdmaServer.cpp
 RdmaServer_LDADD = \
   librdmawrap.la libqpidcommon.la
@@ -318,7 +309,6 @@
   qpid/framing/AMQHeaderBody.cpp \
   qpid/framing/AMQHeartbeatBody.cpp \
   qpid/framing/Array.cpp \
-  qpid/framing/BodyHolder.cpp \
   qpid/framing/BodyHandler.cpp \
   qpid/framing/Buffer.cpp \
   qpid/framing/Endian.cpp \
@@ -335,8 +325,6 @@
   qpid/framing/Proxy.cpp \
   qpid/framing/Uuid.cpp \
   qpid/framing/AMQP_HighestVersion.h \
-  qpid/framing/Blob.cpp \
-  qpid/framing/MaxMethodBodySize.h \
   qpid/framing/TransferContent.cpp \
   qpid/log/Logger.cpp \
   qpid/log/Options.cpp \
@@ -624,9 +612,9 @@
   qpid/framing/AMQP_HighestVersion.h \
   qpid/framing/AccumulatedAck.h \
   qpid/framing/Array.h \
-  qpid/framing/Blob.h \
   qpid/framing/BodyHandler.h \
-  qpid/framing/BodyHolder.h \
+  qpid/framing/BodyFactory.h \
+  qpid/framing/MethodBodyFactory.h \
   qpid/framing/Buffer.h \
   qpid/framing/ChannelHandler.h \
   qpid/framing/Endian.h \
@@ -642,7 +630,6 @@
   qpid/framing/InputHandler.h \
   qpid/framing/InitiationHandler.h \
   qpid/framing/MethodContent.h \
-  qpid/framing/MaxMethodBodySize.h \
   qpid/framing/ModelMethod.h \
   qpid/framing/OutputHandler.h \
   qpid/framing/ProtocolInitiation.h \

Modified: qpid/trunk/qpid/cpp/src/qpid/RefCounted.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/RefCounted.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/RefCounted.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/RefCounted.h Thu Jan 22 12:29:12 2009
@@ -39,11 +39,13 @@
 public:
     RefCounted() : count(0) {}
     void addRef() const { ++count; }
-    void release() const { if (--count==0) delete this; }
+    void release() const { if (--count==0) released(); }
     long refCount() { return count; }
 
 protected:
     virtual ~RefCounted() {};
+    // Allow subclasses to over-ride behavior when refcount reaches 0.
+    virtual void released() const { delete this; }
 };
 
 

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/DeliveryRecord.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/DeliveryRecord.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/DeliveryRecord.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/DeliveryRecord.cpp Thu Jan 22 12:29:12 2009
@@ -93,7 +93,7 @@
         msg.payload->getProperties<framing::DeliveryProperties>()->setRedelivered(true);
     }
 
-    framing::AMQFrame method(framing::in_place<framing::MessageTransferBody>(framing::ProtocolVersion(), tag, acceptExpected ? 0 : 1, acquired ? 0 : 1));
+    framing::AMQFrame method((framing::MessageTransferBody(framing::ProtocolVersion(), tag, acceptExpected ? 0 : 1, acquired ? 0 : 1)));
     method.setEof(false);
     h.handle(method);
     msg.payload->sendHeader(h, framesize);

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/Message.cpp Thu Jan 22 12:29:12 2009
@@ -157,8 +157,7 @@
     if (buffer.available()) {
         //get the data as a string and set that as the content
         //body on a frame then add that frame to the frameset
-        AMQFrame frame;
-        frame.setBody(AMQContentBody());
+        AMQFrame frame((AMQContentBody()));
         frame.castBody<AMQContentBody>()->decode(buffer, buffer.available());
         frames.append(frame);
     } else {
@@ -208,7 +207,7 @@
         bool done = false;
         for (uint64_t offset = 0; !done; offset += maxContentSize)
         {            
-            AMQFrame frame(in_place<AMQContentBody>());
+            AMQFrame frame((AMQContentBody()));
             string& data = frame.castBody<AMQContentBody>()->getData();
 
             store->loadContent(queue, pmsg, data, offset, maxContentSize);

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/MessageBuilder.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/MessageBuilder.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/MessageBuilder.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/MessageBuilder.cpp Thu Jan 22 12:29:12 2009
@@ -49,8 +49,7 @@
         if (type == CONTENT_BODY) {
             //TODO: rethink how to handle non-existent headers(?)...
             //didn't get a header: add in a dummy
-            AMQFrame header;
-            header.setBody(AMQHeaderBody());
+            AMQFrame header((AMQHeaderBody()));
             header.setBof(false);
             header.setEof(false);
             message->getFrames().append(header);            

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SessionState.cpp Thu Jan 22 12:29:12 2009
@@ -188,8 +188,7 @@
     if (frame.getEof() && frame.getEos()) {//end of frameset
         if (frame.getBof()) {
             //i.e this is a just a command frame, add a dummy header
-            AMQFrame header;
-            header.setBody(AMQHeaderBody());
+            AMQFrame header((AMQHeaderBody()));
             header.setBof(false);
             header.setEof(false);
             msg->getFrames().append(header);                        

Modified: qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp Thu Jan 22 12:29:12 2009
@@ -164,7 +164,7 @@
 
 void ConnectionImpl::idleOut()
 {
-    AMQFrame frame(in_place<AMQHeartbeatBody>());
+    AMQFrame frame((AMQHeartbeatBody()));
     connector->send(frame);
 }
 

Modified: qpid/trunk/qpid/cpp/src/qpid/client/SessionImpl.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/SessionImpl.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/SessionImpl.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/SessionImpl.cpp Thu Jan 22 12:29:12 2009
@@ -364,7 +364,7 @@
         const uint32_t frag_size = maxFrameSize - AMQFrame::frameOverhead(); 
 
         if(data_length < frag_size){
-            AMQFrame frame(in_place<AMQContentBody>(content.getData()));
+            AMQFrame frame((AMQContentBody(content.getData())));
             frame.setFirstSegment(false);
             handleOut(frame);
         }else{
@@ -373,7 +373,7 @@
             while (remaining > 0) {
                 uint32_t length = remaining > frag_size ? frag_size : remaining;
                 string frag(content.getData().substr(offset, length));
-                AMQFrame frame(in_place<AMQContentBody>(frag));
+                AMQFrame frame((AMQContentBody(frag)));
                 frame.setFirstSegment(false);
                 frame.setLastSegment(true);
                 if (offset > 0) {

Modified: qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp Thu Jan 22 12:29:12 2009
@@ -109,7 +109,7 @@
                 QPID_LOG(debug, cluster << " inserting connection " << *this);
                 cluster.insert(boost::intrusive_ptr<Connection>(this));
             }
-            AMQFrame ok(in_place<ConnectionCloseOkBody>());
+            AMQFrame ok((ConnectionCloseOkBody()));
             connection.getOutput().send(ok);
             output.setOutputHandler(discardHandler);
             catchUp = false;

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h Thu Jan 22 12:29:12 2009
@@ -22,7 +22,9 @@
  *
  */
 #include "qpid/framing/amqp_types.h"
-
+#include "qpid/RefCounted.h"
+#include "qpid/framing/BodyFactory.h"
+#include <boost/intrusive_ptr.hpp>
 #include <ostream>
 
 namespace qpid {
@@ -43,11 +45,15 @@
     virtual void visit(const AMQMethodBody&) = 0;
 };
 
-class AMQBody
-{
+class AMQBody : public RefCounted {
   public:
+    AMQBody() {}
     virtual ~AMQBody();
 
+    // Make AMQBody copyable even though RefCounted. 
+    AMQBody(const AMQBody&) : RefCounted() {}  
+    AMQBody& operator=(const AMQBody&) { return *this; }
+
     virtual uint8_t type() const = 0;
 
     virtual void encode(Buffer& buffer) const = 0;
@@ -62,6 +68,7 @@
 
     /** Match if same type and same class/method ID for methods */
     static bool match(const AMQBody& , const AMQBody& );
+    virtual boost::intrusive_ptr<AMQBody> clone() const = 0;
 };
 
 std::ostream& operator<<(std::ostream& out, const AMQBody& body) ;

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQContentBody.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQContentBody.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQContentBody.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQContentBody.h Thu Jan 22 12:29:12 2009
@@ -44,6 +44,7 @@
     void decode(Buffer& buffer, uint32_t size);
     void print(std::ostream& out) const;
     void accept(AMQBodyConstVisitor& v) const { v.visit(*this); }
+    boost::intrusive_ptr<AMQBody> clone() const { return BodyFactory::copy(*this); }
 };
 
 }

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.cpp Thu Jan 22 12:29:12 2009
@@ -20,25 +20,31 @@
  */
 #include "AMQFrame.h"
 
-#include "qpid/framing/variant.h"
 #include "qpid/framing/AMQMethodBody.h"
 #include "qpid/framing/reply_exceptions.h"
-
+#include "qpid/framing/BodyFactory.h"
+#include "qpid/framing/MethodBodyFactory.h"
 #include <boost/format.hpp>
-
 #include <iostream>
 
 namespace qpid {
 namespace framing {
 
-AMQFrame::~AMQFrame() {}
+void AMQFrame::init() { bof = eof = bos = eos = true; subchannel=0; channel=0; }
+
+AMQFrame::AMQFrame(const boost::intrusive_ptr<AMQBody>& b) : body(b) { init(); }
+
+AMQFrame::AMQFrame(const AMQBody& b) : body(b.clone()) { init(); }
 
-void AMQFrame::setBody(const AMQBody& b) { body = new BodyHolder(b); }
+AMQFrame::~AMQFrame() {}
 
-void AMQFrame::setMethod(ClassId c, MethodId m) { body = new BodyHolder(c,m); }
+void AMQFrame::setMethod(ClassId c, MethodId m) { body = MethodBodyFactory::create(c,m); }
 
 uint32_t AMQFrame::encodedSize() const {
-    return frameOverhead() + body->encodedSize();
+    uint32_t size = frameOverhead() + body->encodedSize();
+    if (body->getMethod())
+        size +=  sizeof(ClassId)+sizeof(MethodId);
+    return size;
 }
 
 uint32_t AMQFrame::frameOverhead() {
@@ -65,6 +71,11 @@
     buffer.putOctet(0x0f & track);
     buffer.putShort(channel);    
     buffer.putLong(0);
+    const AMQMethodBody* method=getMethod();
+    if (method) {
+        buffer.putOctet(method->amqpClassId());
+        buffer.putOctet(method->amqpMethodId());
+    }
     body->encode(buffer);
 }
 
@@ -105,8 +116,24 @@
         buffer.restore();
         return false;
     }
-    body = new BodyHolder();
-    body->decode(type,buffer, body_size);
+
+    switch(type)
+    {
+      case 0://CONTROL 
+      case METHOD_BODY: {
+          ClassId c = buffer.getOctet();
+          MethodId m = buffer.getOctet();
+          body = MethodBodyFactory::create(c, m);
+          break;
+      }
+      case HEADER_BODY: body =  BodyFactory::create<AMQHeaderBody>(); break;
+      case CONTENT_BODY: body = BodyFactory::create<AMQContentBody>(); break;
+      case HEARTBEAT_BODY: body = BodyFactory::create<AMQHeartbeatBody>(); break;
+      default:
+	throw IllegalArgumentException(QPID_MSG("Invalid frame type " << type));
+    }
+    body->decode(buffer, body_size);
+
     return true;
 }
 

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQFrame.h Thu Jan 22 12:29:12 2009
@@ -26,47 +26,29 @@
 #include "AMQContentBody.h"
 #include "AMQHeartbeatBody.h"
 #include "ProtocolVersion.h"
-#include "BodyHolder.h"
 #include "qpid/sys/LatencyMetric.h"
-
 #include <boost/intrusive_ptr.hpp>
 #include <boost/cast.hpp>
 
 namespace qpid {
 namespace framing {
 
-class BodyHolder;
-
 class AMQFrame : public AMQDataBlock, public sys::LatencyMetricTimestamp
 {
   public:
-    AMQFrame(boost::intrusive_ptr<BodyHolder> b=0) : body(b) { init(); }
-    AMQFrame(const AMQBody& b) { setBody(b); init(); }
+    AMQFrame(const boost::intrusive_ptr<AMQBody>& b=0);
+    AMQFrame(const AMQBody& b);
     ~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; }
 
-    boost::intrusive_ptr<BodyHolder> getHolder() { return body; }
-    
-    AMQBody* getBody() { return body ? body->get() : 0; }
-    const AMQBody* getBody() const { return body ? body->get() : 0; }
+    AMQBody* getBody() { return body.get(); }
+    const AMQBody* getBody() const { return body.get(); }
 
     AMQMethodBody* getMethod() { return getBody()->getMethod(); }
     const AMQMethodBody* getMethod() const { return getBody()->getMethod(); }
 
-    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);
 
     template <class T> T* castBody() {
@@ -109,10 +91,11 @@
     static uint32_t frameOverhead();
     /** Must point to at least DECODE_SIZE_MIN bytes of data */
     static uint16_t decodeSize(char* data);
+
   private:
-    void init() { bof = eof = bos = eos = true; subchannel=0; channel=0; }
+    void init();
 
-    boost::intrusive_ptr<BodyHolder> body;
+    boost::intrusive_ptr<AMQBody> body;
     uint16_t channel : 16;
     uint8_t subchannel : 8;
     bool bof : 1;

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeaderBody.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeaderBody.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeaderBody.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeaderBody.h Thu Jan 22 12:29:12 2009
@@ -99,6 +99,8 @@
     template <class T> const T* get() const {
         return properties.OptProps<T>::props.get_ptr();
     }
+
+    boost::intrusive_ptr<AMQBody> clone() const { return BodyFactory::copy(*this); }
 };
 
 }}

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeartbeatBody.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeartbeatBody.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeartbeatBody.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/AMQHeartbeatBody.h Thu Jan 22 12:29:12 2009
@@ -38,6 +38,7 @@
     inline void decode(Buffer& , uint32_t /*size*/) {}
     virtual void print(std::ostream& out) const;
     void accept(AMQBodyConstVisitor& v) const { v.visit(*this); }
+    boost::intrusive_ptr<AMQBody> clone() const { return BodyFactory::copy(*this); }
 };
 
 }

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/Array.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/Array.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/Array.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/Array.cpp Thu Jan 22 12:29:12 2009
@@ -75,6 +75,7 @@
 }
 
 void Array::decode(Buffer& buffer){
+    values.clear();
     uint32_t size = buffer.getLong();//size added only when array is a top-level type
     uint32_t available = buffer.available();
     if (available < size) {

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/Blob.h Thu Jan 22 12:29:12 2009
@@ -1,197 +0,0 @@
-#ifndef QPID_FRAMING_BLOB_H
-#define QPID_FRAMING_BLOB_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 <boost/static_assert.hpp>
-#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 <boost/version.hpp>
-
-#include <new>
-
-#include <assert.h>
-
-
-namespace qpid {
-namespace framing {
-
-using boost::in_place;          
-using boost::typed_in_place_factory_base;
-
-/** 0-arg typed_in_place_factory, missing in pre-1.35 boost. */
-#if (BOOST_VERSION < 103500)
-template <class T>
-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>(); }
-#endif
-
-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) {
-        new (dest) T(*static_cast<const T*>(src));
-    }
-};
-
-template <> struct BlobHelper<void> {
-    static void destroy(void*);
-    static void copy(void* to, const void* from);
-};
-
-/**
- * A Blob is a chunk of memory which can contain a single object at
- * a time-arbitrary type, provided sizeof(T)<=blob.size(). Using Blobs
- * ensures proper construction and destruction of its contents,
- * and proper copying between Blobs, but nothing else.
- * 
- * In particular you must ensure that the Blob is big enough for its
- * contents and must know the type of object in the Blob to cast get().
- *
- * If BaseType is specified then only an object that can be
- * static_cast to BaseType may be stored in the Blob.
- */
-template <size_t Size, class BaseType=void>
-class Blob
-{
-    boost::aligned_storage<Size> store;
-    BaseType* basePtr;
-    
-    void (*destroy)(void*);
-    void (*copy)(void*, const void*);
-
-    template <class T>void setType() {
-        BOOST_STATIC_ASSERT(sizeof(T) <= Size);
-        destroy=&BlobHelper<T>::destroy;
-        copy=&BlobHelper<T>::copy;
-        // Base pointer may be offeset from store.address()
-        basePtr = reinterpret_cast<T*>(store.address());
-    }
-        
-    void initialize() {
-        destroy=&BlobHelper<void>::destroy;
-        copy=&BlobHelper<void>::copy;
-        basePtr=0;
-    }
-
-    template<class Factory>
-    typename EnableInPlace<Factory>::type apply(const Factory& factory)
-    {
-        typedef typename Factory::value_type T;
-        assert(empty());
-        factory.apply(store.address());
-        setType<T>();
-    }
-
-    void assign(const Blob& b) {
-        assert(empty());
-        if (b.empty()) return;
-        b.copy(this->store.address(), b.store.address());
-        copy = b.copy;
-        destroy = b.destroy;
-        basePtr = reinterpret_cast<BaseType*>(
-            ((char*)this)+ ((const char*)(b.basePtr) - (const char*)(&b)));
-    }
-
-  public:
-    /** Construct an empty Blob. */
-    Blob() { initialize(); }
-
-    /** Copy a Blob. */
-    Blob(const Blob& b) { initialize(); assign(b); }
-
-    /** Construct from in_place constructor. */
-    template<class InPlace>
-    Blob(const InPlace & expr, typename EnableInPlace<InPlace>::type* =0) {
-        initialize(); apply(expr);
-    }
-
-    /** Construct by copying an object constructor. */
-    template<class T>
-    Blob(const T & t, typename DisableInPlace<T>::type* =0) {
-        initialize(); apply(in_place<T>(t));
-    }
-
-    ~Blob() { clear(); }
-
-    /** Assign from another Blob. */
-    Blob& operator=(const Blob& b) {
-        clear();
-        assign(b);
-        return *this;
-    }
-
-    /** Assign from an in_place constructor expression. */
-    template<class InPlace>
-    typename EnableInPlace<InPlace,Blob&>::type operator=(const InPlace& expr) {
-        clear(); apply(expr); return *this;
-    }
-
-    /** Assign from an object of type T. */
-    template <class T>
-    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; }
-
-    /** Get pointer to Blob contents, returns 0 if empty. */
-    const BaseType* get() const { return basePtr; }
-
-    /** Destroy the object in the Blob making it empty. */
-    void clear() {
-        void (*oldDestroy)(void*) = destroy; 
-        initialize();
-        oldDestroy(store.address());
-    }
-
-    bool empty() const { return destroy==BlobHelper<void>::destroy; }
-    
-    static size_t size() { return Size; }
-};
-
-}} // namespace qpid::framing
-
-
-#endif  /*!QPID_FRAMING_BLOB_H*/

Added: qpid/trunk/qpid/cpp/src/qpid/framing/BodyFactory.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/BodyFactory.h?rev=736783&view=auto
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/BodyFactory.h (added)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/BodyFactory.h Thu Jan 22 12:29:12 2009
@@ -0,0 +1,47 @@
+#ifndef QPID_FRAMING_BODYFACTORY_H
+#define QPID_FRAMING_BODYFACTORY_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 <boost/intrusive_ptr.hpp>
+
+namespace qpid {
+namespace framing {
+
+/**
+ * Indirect creation of body types to allow centralized changes to
+ * memory management strategy.
+ */
+class BodyFactory {
+  public:
+    template <class BodyType> static boost::intrusive_ptr<BodyType> create() {
+        return new BodyType;
+    }
+
+    template <class BodyType> static boost::intrusive_ptr<BodyType> copy(const BodyType& body) {
+        return new BodyType(body);
+    }
+};
+
+}} // namespace qpid::framing
+
+#endif  /*!QPID_FRAMING_BODYFACTORY_H*/

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

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

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.cpp Thu Jan 22 12:29:12 2009
@@ -1,76 +0,0 @@
-/*
- *
- * 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 0://CONTROL 
-      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 IllegalArgumentException(QPID_MSG("Invalid frame type " << type));
-    }
-    get()->decode(buffer, size);
-}
-
-uint32_t BodyHolder::encodedSize() const {
-    const AMQMethodBody* method=getMethod();
-    if (method) 
-        return sizeof(ClassId)+sizeof(MethodId)+method->encodedSize();
-    else
-        return get()->encodedSize();
-}
-
-}} // namespace qpid::framing
-

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/BodyHolder.h Thu Jan 22 12:29:12 2009
@@ -1,88 +0,0 @@
-#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 encodedSize() 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*/

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/FieldTable.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/FieldTable.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/FieldTable.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/FieldTable.cpp Thu Jan 22 12:29:12 2009
@@ -201,6 +201,7 @@
 }
 
 void FieldTable::decode(Buffer& buffer){
+    clear();
     uint32_t len = buffer.getLong();
     if (len) {
         uint32_t available = buffer.available();

Added: qpid/trunk/qpid/cpp/src/qpid/framing/MethodBodyFactory.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/MethodBodyFactory.h?rev=736783&view=auto
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/MethodBodyFactory.h (added)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/MethodBodyFactory.h Thu Jan 22 12:29:12 2009
@@ -0,0 +1,44 @@
+#ifndef QPID_FRAMING_METHODBODYFACTORY_H
+#define QPID_FRAMING_METHODBODYFACTORY_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 "amqp_types.h"
+#include <boost/intrusive_ptr.hpp>
+
+namespace qpid {
+namespace framing {
+
+class AMQMethodBody;
+
+/**
+ * Functions to create instances of AMQMethodBody sub-classes.
+ * Note: MethodBodyFactory.cpp file is generated by rubygen.
+ */
+class MethodBodyFactory
+{
+  public:
+    static boost::intrusive_ptr<AMQMethodBody> create(ClassId c, MethodId m);
+};
+
+}} // namespace qpid::framing
+
+#endif  /*!QPID_FRAMING_METHODBODYFACTORY_H*/

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

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

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/SendContent.cpp Thu Jan 22 12:29:12 2009
@@ -53,8 +53,7 @@
 
 void qpid::framing::SendContent::sendFragment(const AMQContentBody& body, uint32_t offset, uint16_t size, bool first, bool last) const
 {
-    AMQFrame fragment(in_place<AMQContentBody>(
-                          body.getData().substr(offset, size)));
+    AMQFrame fragment((AMQContentBody(body.getData().substr(offset, size))));
     setFlags(fragment, first, last);
     handler.handle(fragment);
 }

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/SequenceNumberSet.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/SequenceNumberSet.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/SequenceNumberSet.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/SequenceNumberSet.cpp Thu Jan 22 12:29:12 2009
@@ -33,6 +33,7 @@
 
 void SequenceNumberSet::decode(Buffer& buffer)
 {
+    clear();
     uint16_t count = (buffer.getShort() / 4);
     for (uint16_t i = 0; i < count; i++) {
         push_back(SequenceNumber(buffer.getLong()));

Modified: qpid/trunk/qpid/cpp/src/qpid/framing/SequenceSet.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/framing/SequenceSet.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/framing/SequenceSet.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/framing/SequenceSet.cpp Thu Jan 22 12:29:12 2009
@@ -46,6 +46,7 @@
 
 void SequenceSet::decode(Buffer& buffer)
 {
+    clear();
     uint16_t size = buffer.getShort();
     uint16_t count = size / RANGE_SIZE;//number of ranges
     if (size % RANGE_SIZE)

Modified: qpid/trunk/qpid/cpp/src/qpid/management/ManagementBroker.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/management/ManagementBroker.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/management/ManagementBroker.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/management/ManagementBroker.cpp Thu Jan 22 12:29:12 2009
@@ -301,10 +301,9 @@
         return;
 
     intrusive_ptr<Message> msg(new Message());
-    AMQFrame method(in_place<MessageTransferBody>(
-        ProtocolVersion(), exchange->getName (), 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
-    AMQFrame content(in_place<AMQContentBody>());
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange->getName (), 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
+    AMQFrame content((AMQContentBody()));
 
     content.castBody<AMQContentBody>()->decode(buf, length);
 

Modified: qpid/trunk/qpid/cpp/src/qpid/replication/ReplicatingEventListener.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/replication/ReplicatingEventListener.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/replication/ReplicatingEventListener.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/replication/ReplicatingEventListener.cpp Thu Jan 22 12:29:12 2009
@@ -77,8 +77,8 @@
 boost::intrusive_ptr<Message> ReplicatingEventListener::createMessage(const FieldTable& headers)
 {
     boost::intrusive_ptr<Message> msg(new Message());
-    AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), EMPTY, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), EMPTY, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
     header.setBof(false);
     header.setEof(true);
     header.setBos(true);
@@ -105,7 +105,7 @@
 boost::intrusive_ptr<Message> ReplicatingEventListener::cloneMessage(Queue& queue, boost::intrusive_ptr<Message> original)
 {
     boost::intrusive_ptr<Message> copy(new Message());
-    AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), EMPTY, 0, 0));
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), EMPTY, 0, 0)));
     AppendingHandler handler(copy);
     handler.handle(method);
     original->sendHeader(handler, std::numeric_limits<int16_t>::max());

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/Thread.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/Thread.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/Thread.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/Thread.h Thu Jan 22 12:29:12 2009
@@ -25,10 +25,10 @@
 
 #ifdef _WIN32
 #  define QPID_TSS __declspec(thread)
-#elif defined (gcc)
+#elif defined (__GNUC__)
 #  define QPID_TSS __thread
 #else
-#  define QPID_TSS
+#  error "Dont know how to define QPID_TSS for this platform"
 #endif
 
 namespace qpid {

Modified: qpid/trunk/qpid/cpp/src/tests/Blob.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/Blob.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/Blob.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/Blob.cpp Thu Jan 22 12:29:12 2009
@@ -1,128 +0,0 @@
-/*
- * 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/Blob.h"
-
-#include "unit_test.h"
-
-QPID_AUTO_TEST_SUITE(BlobTestSuite)
-
-using namespace std;
-using namespace qpid::framing;
-
-struct Base {
-    int id;
-    int magic;
-
-    Base(int n) : id(n), magic(42) {}
-    Base(const Base& c) : id(c.id), magic(42) {}
-    ~Base() { BOOST_CHECK_EQUAL(42, magic); } // Detect random data. 
-};
-
-template <class T> struct Count : public Base {
-    static int instances;
-    bool destroyed;
-    
-    Count(int n) : Base(n), destroyed(false) { ++instances; }
-    Count(const Count& c) : Base(c), destroyed(false) { ++instances; }
-    ~Count() {
-        BOOST_CHECK(!destroyed); // Detect double-destructor
-        destroyed=true;
-        BOOST_CHECK(--instances >= 0);
-    }
-};
-
-template <class T> int Count<T>::instances = 0;
-
-struct Foo : public Count<Foo> { Foo(int n) : Count<Foo>(n) {}; };
-struct Bar : public Count<Bar> { Bar(int n) : Count<Bar>(n) {}; };
-
-typedef Blob<sizeof(Foo), Base> TestBlob ;
-
-QPID_AUTO_TEST_CASE(testBlobCtor) {
-    {
-        TestBlob empty;
-        BOOST_CHECK(empty.empty());
-        BOOST_CHECK(empty.get() == 0);
-
-        TestBlob empty2(empty);
-        BOOST_CHECK(empty2.empty());
-
-        TestBlob foo(in_place<Foo>(1));
-        BOOST_CHECK(!foo.empty());
-        BOOST_CHECK_EQUAL(1, foo.get()->id);
-        BOOST_CHECK_EQUAL(1, Foo::instances);
-
-        TestBlob foo2(foo);
-        BOOST_CHECK(!foo2.empty());
-        BOOST_CHECK_EQUAL(1, foo2.get()->id);
-        BOOST_CHECK_EQUAL(2, Foo::instances);
-    }
-    
-    BOOST_CHECK_EQUAL(0, Foo::instances);
-    BOOST_CHECK_EQUAL(0, Bar::instances);
-}
-
-
-QPID_AUTO_TEST_CASE(testAssign) {
-    {
-        TestBlob b;
-        b = Foo(2);
-        BOOST_CHECK_EQUAL(2, b.get()->id);
-        BOOST_CHECK_EQUAL(1, Foo::instances);
-
-        TestBlob b2(b);
-        BOOST_CHECK_EQUAL(2, b.get()->id);
-        BOOST_CHECK_EQUAL(2, Foo::instances);
-    
-        b2 = Bar(3);
-        BOOST_CHECK_EQUAL(3, b2.get()->id);
-        BOOST_CHECK_EQUAL(1, Foo::instances);
-        BOOST_CHECK_EQUAL(1, Bar::instances);
-
-        b2 = in_place<Foo>(4); 
-        BOOST_CHECK_EQUAL(4, b2.get()->id);
-        BOOST_CHECK_EQUAL(2, Foo::instances);
-        BOOST_CHECK_EQUAL(0, Bar::instances);
-
-        b2.clear();
-        BOOST_CHECK(b2.empty());
-        BOOST_CHECK_EQUAL(1, Foo::instances);
-    }
-    BOOST_CHECK_EQUAL(0, Foo::instances);
-    BOOST_CHECK_EQUAL(0, Bar::instances);
-}
-
-
-QPID_AUTO_TEST_CASE(testClear) {
-    TestBlob b(in_place<Foo>(5));
-    TestBlob c(b);
-    BOOST_CHECK(!c.empty());
-    BOOST_CHECK(!b.empty());
-    BOOST_CHECK_EQUAL(2, Foo::instances);
-
-    c.clear();
-    BOOST_CHECK(c.empty());
-    BOOST_CHECK_EQUAL(1, Foo::instances);
-
-    b.clear(); 
-    BOOST_CHECK(b.empty());
-    BOOST_CHECK_EQUAL(0, Foo::instances);
-}
-
-QPID_AUTO_TEST_SUITE_END()

Modified: qpid/trunk/qpid/cpp/src/tests/ExchangeTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/ExchangeTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/ExchangeTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/ExchangeTest.cpp Thu Jan 22 12:29:12 2009
@@ -167,8 +167,8 @@
 
 intrusive_ptr<Message> cmessage(std::string exchange, std::string routingKey) {
     intrusive_ptr<Message> msg(new Message());
-    AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
     msg->getFrames().append(method);
     msg->getFrames().append(header);
     msg->getFrames().getHeaders()->get<DeliveryProperties>(true)->setRoutingKey(routingKey);

Modified: qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/FramingTest.cpp Thu Jan 22 12:29:12 2009
@@ -123,7 +123,7 @@
     hosts.add(boost::shared_ptr<FieldValue>(new Str16Value(a)));
     hosts.add(boost::shared_ptr<FieldValue>(new Str16Value(b)));
         
-    AMQFrame in(in_place<ConnectionRedirectBody>(version, a, hosts));
+    AMQFrame in((ConnectionRedirectBody(version, a, hosts)));
     in.setChannel(999);
     in.encode(wbuff);
 
@@ -138,7 +138,7 @@
     char buffer[1024];
     ProtocolVersion version(highestProtocolVersion);
     Buffer wbuff(buffer, sizeof(buffer));
-    AMQFrame in(in_place<MessageCancelBody>(version, "tag"));
+    AMQFrame in((MessageCancelBody(version, "tag")));
     in.setChannel(999);
     in.encode(wbuff);
 

Modified: qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/HeaderTest.cpp Thu Jan 22 12:29:12 2009
@@ -49,7 +49,7 @@
 
 QPID_AUTO_TEST_CASE(testMessageProperties) 
 {
-    AMQFrame out(in_place<AMQHeaderBody>());
+    AMQFrame out((AMQHeaderBody()));
     MessageProperties* props1 = 
         out.castBody<AMQHeaderBody>()->get<MessageProperties>(true);
 
@@ -84,7 +84,7 @@
 
 QPID_AUTO_TEST_CASE(testDeliveryProperies) 
 {
-    AMQFrame out(in_place<AMQHeaderBody>());
+    AMQFrame out((AMQHeaderBody()));
     DeliveryProperties* props1 = 
         out.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true);
 

Modified: qpid/trunk/qpid/cpp/src/tests/Makefile.am
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/Makefile.am?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/Makefile.am (original)
+++ qpid/trunk/qpid/cpp/src/tests/Makefile.am Thu Jan 22 12:29:12 2009
@@ -54,7 +54,7 @@
 	BrokerFixture.h SocketProxy.h \
 	exception_test.cpp \
 	RefCounted.cpp \
-	SessionState.cpp Blob.cpp logging.cpp \
+	SessionState.cpp logging.cpp \
 	AsyncCompletion.cpp \
 	Url.cpp Uuid.cpp \
 	Shlib.cpp FieldValue.cpp FieldTable.cpp Array.cpp \

Modified: qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/MessageBuilderTest.cpp Thu Jan 22 12:29:12 2009
@@ -101,9 +101,8 @@
     std::string exchange("builder-exchange");
     std::string key("builder-exchange");
 
-    AMQFrame method(in_place<MessageTransferBody>(
-                        ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
 
     header.castBody<AMQHeaderBody>()->get<MessageProperties>(true)->setContentLength(0);        
     header.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true)->setRoutingKey(key);
@@ -126,9 +125,9 @@
     std::string exchange("builder-exchange");
     std::string key("builder-exchange");
 
-    AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
-    AMQFrame content(in_place<AMQContentBody>(data));
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
+    AMQFrame content((AMQContentBody(data)));
     method.setEof(false);
     header.setBof(false);
     header.setEof(false);
@@ -160,11 +159,10 @@
     std::string exchange("builder-exchange");
     std::string key("builder-exchange");
 
-    AMQFrame method(in_place<MessageTransferBody>(
-                        ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
-    AMQFrame content1(in_place<AMQContentBody>(data1));
-    AMQFrame content2(in_place<AMQContentBody>(data2));
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
+    AMQFrame content1((AMQContentBody(data1)));
+    AMQFrame content2((AMQContentBody(data2)));
     method.setEof(false);
     header.setBof(false);
     header.setEof(false);
@@ -197,11 +195,10 @@
     std::string exchange("builder-exchange");
     std::string key("builder-exchange");
 
-    AMQFrame method(in_place<MessageTransferBody>(
-                        ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
-    AMQFrame content1(in_place<AMQContentBody>(data1));
-    AMQFrame content2(in_place<AMQContentBody>(data2));
+    AMQFrame method(MessageTransferBody(ProtocolVersion(), exchange, 0, 0));
+    AMQFrame header((AMQHeaderBody()));
+    AMQFrame content1((AMQContentBody(data1)));
+    AMQFrame content2((AMQContentBody(data2)));
 
     header.castBody<AMQHeaderBody>()->get<MessageProperties>(true)->setContentLength(data1.size() + data2.size());        
     header.castBody<AMQHeaderBody>()->get<DeliveryProperties>(true)->setRoutingKey(key);

Modified: qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/MessageTest.cpp Thu Jan 22 12:29:12 2009
@@ -46,11 +46,10 @@
 
     intrusive_ptr<Message> msg(new Message());
 
-    AMQFrame method(in_place<MessageTransferBody>(
-                        ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
-    AMQFrame content1(in_place<AMQContentBody>(data1));
-    AMQFrame content2(in_place<AMQContentBody>(data2));
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
+    AMQFrame content1((AMQContentBody(data1)));
+    AMQFrame content2((AMQContentBody(data2)));
 
     msg->getFrames().append(method);
     msg->getFrames().append(header);

Modified: qpid/trunk/qpid/cpp/src/tests/MessageUtils.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/MessageUtils.h?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/MessageUtils.h (original)
+++ qpid/trunk/qpid/cpp/src/tests/MessageUtils.h Thu Jan 22 12:29:12 2009
@@ -35,8 +35,8 @@
     {
         boost::intrusive_ptr<Message> msg(new Message());
 
-        AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), exchange, 0, 0));
-        AMQFrame header(in_place<AMQHeaderBody>());
+        AMQFrame method(( MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+        AMQFrame header((AMQHeaderBody()));
 
         msg->getFrames().append(method);
         msg->getFrames().append(header);
@@ -49,7 +49,7 @@
 
     static void addContent(boost::intrusive_ptr<Message> msg, const string& data)
     {
-        AMQFrame content(in_place<AMQContentBody>(data));
+        AMQFrame content((AMQContentBody(data)));
         msg->getFrames().append(content);
     }
 };

Modified: qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/QueueTest.cpp Thu Jan 22 12:29:12 2009
@@ -67,8 +67,8 @@
 
 intrusive_ptr<Message> create_message(std::string exchange, std::string routingKey) {
     intrusive_ptr<Message> msg(new Message());
-    AMQFrame method(in_place<MessageTransferBody>(ProtocolVersion(), exchange, 0, 0));
-    AMQFrame header(in_place<AMQHeaderBody>());
+    AMQFrame method((MessageTransferBody(ProtocolVersion(), exchange, 0, 0)));
+    AMQFrame header((AMQHeaderBody()));
     msg->getFrames().append(method);
     msg->getFrames().append(header);
     msg->getFrames().getHeaders()->get<DeliveryProperties>(true)->setRoutingKey(routingKey);

Modified: qpid/trunk/qpid/cpp/src/tests/SessionState.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/SessionState.cpp?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/SessionState.cpp (original)
+++ qpid/trunk/qpid/cpp/src/tests/SessionState.cpp Thu Jan 22 12:29:12 2009
@@ -45,8 +45,7 @@
 
 // Create a frame with a one-char string.
 AMQFrame& frame(char s) {
-    static AMQFrame frame;
-    frame.setBody(AMQContentBody(string(&s, 1)));
+    static AMQFrame frame((AMQContentBody(string(&s, 1))));
     return frame;
 }
 
@@ -64,7 +63,7 @@
 }
 // Make a transfer command frame.
 AMQFrame transferFrame(bool hasContent) {
-    AMQFrame t(in_place<MessageTransferBody>());
+    AMQFrame t((MessageTransferBody()));
     t.setFirstFrame(true);
     t.setLastFrame(true);
     t.setFirstSegment(true);
@@ -73,7 +72,7 @@
 }
 // Make a content frame
 AMQFrame contentFrame(string content, bool isLast=true) {
-    AMQFrame f(in_place<AMQContentBody>(content));
+    AMQFrame f((AMQContentBody(content)));
     f.setFirstFrame(true);
     f.setLastFrame(true);
     f.setFirstSegment(false);
@@ -116,8 +115,8 @@
                            bind(transfer1Char, ref(s), _1));
 }
 
-size_t contentFrameSize(size_t n=1) { return AMQFrame(in_place<AMQContentBody>()).encodedSize() + n; }
-size_t transferFrameSize() { return AMQFrame(in_place<MessageTransferBody>()).encodedSize(); }
+size_t contentFrameSize(size_t n=1) { return AMQFrame(( AMQContentBody())).encodedSize() + n; }
+size_t transferFrameSize() { return AMQFrame((MessageTransferBody())).encodedSize(); }
 
 // ==== qpid::SessionState test classes
 
@@ -134,7 +133,7 @@
     transferN(s, "xyz");
     BOOST_CHECK_EQUAL(str(s.senderExpected(SessionPoint(0,0))),"CabcCdCeCfCxyz");
     // Ignore controls.
-    s.senderRecord(AMQFrame(in_place<SessionFlushBody>()));
+    s.senderRecord(AMQFrame(new SessionFlushBody()));
     BOOST_CHECK_EQUAL(str(s.senderExpected(SessionPoint(2,0))),"CeCfCxyz");    
 }
 

Modified: qpid/trunk/qpid/cpp/src/tests/ssl_test
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/tests/ssl_test?rev=736783&r1=736782&r2=736783&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/tests/ssl_test (original)
+++ qpid/trunk/qpid/cpp/src/tests/ssl_test Thu Jan 22 12:29:12 2009
@@ -25,7 +25,7 @@
 CERT_DIR=`pwd`/test_cert_db
 CERT_PW_FILE=`pwd`/cert.password
 HOSTNAME=`hostname`
-COUNT=10000
+COUNT=10
 
 trap stop_broker EXIT
 



---------------------------------------------------------------------
Apache Qpid - AMQP Messaging Implementation
Project:      http://qpid.apache.org
Use/Interact: mailto:commits-subscribe@qpid.apache.org