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/06/27 23:19:15 UTC

svn commit: r551336 - in /incubator/qpid/trunk/qpid/cpp/src: ./ qpid/broker/ qpid/client/ qpid/framing/ qpid/sys/apr/ tests/

Author: aconway
Date: Wed Jun 27 14:19:14 2007
New Revision: 551336

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

	* src/qpid/framing/ChannelAdapter.cpp: Use handler chains
	for in and outbound frames.

	* src/qpid/framing/InputHandler.h, OutputHandler.h, FrameHandler.h:
	All handlers pass AMQFrame& and have consistent memory management.
	Terminal OutputHandlers used to take ownership and delete frame, now
	they make a shallow copy instead.

	* src/qpid/framing/Handler.h, FrameHandler.h: Simplified.

	* src/qpid/client/ClientConnection.cpp: 
	* src/qpid/broker/Connection.cpp: 
	* src/qpid/broker/BrokerChannel.cpp:
	   Update for ChannelAdapter changes.

Removed:
    incubator/qpid/trunk/qpid/cpp/src/tests/Handler.cpp
Modified:
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerChannel.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/ClientConnection.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connection.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameHandler.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Handler.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/sys/apr/LFSessionContext.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.h
    incubator/qpid/trunk/qpid/cpp/src/qpidd.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/InProcessBroker.h
    incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerChannel.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerChannel.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerChannel.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/BrokerChannel.cpp Wed Jun 27 14:19:14 2007
@@ -55,7 +55,7 @@
     uint32_t _framesize, MessageStore* const _store,
     uint64_t _stagingThreshold
 ) :
-    ChannelAdapter(id, &con.getOutput(), con.getVersion()),
+    ChannelAdapter(),
     connection(con),
     currentDeliveryTag(1),
     prefetchSize(0),
@@ -70,6 +70,7 @@
     flowActive(true),
     adapter(new BrokerAdapter(*this, con, con.broker))
 {
+    init(id, con.getOutput(), con.getVersion());
     outstanding.reset();
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp Wed Jun 27 14:19:14 2007
@@ -66,8 +66,8 @@
 }
 
 
-void Connection::received(framing::AMQFrame* frame){
-    getChannel(frame->getChannel()).handleBody(frame->getBody());
+void Connection::received(framing::AMQFrame& frame){
+    getChannel((frame.getChannel())).getHandlers().in->handle(frame);
 }
 
 void Connection::close(

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h Wed Jun 27 14:19:14 2007
@@ -80,7 +80,7 @@
     std::vector<Queue::shared_ptr> exclusiveQueues;
 
     // ConnectionInputHandler methods
-    void received(framing::AMQFrame* frame);
+    void received(framing::AMQFrame& frame);
     void initiated(const framing::ProtocolInitiation& header);
     void idleOut();
     void idleIn();

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/ClientConnection.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/ClientConnection.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/ClientConnection.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/ClientConnection.cpp Wed Jun 27 14:19:14 2007
@@ -133,25 +133,22 @@
     channels.erase(id);
 }
 
-void Connection::received(AMQFrame* frame){
-    // FIXME aconway 2007-01-25: Mutex 
-    ChannelId id = frame->getChannel();
+void Connection::received(AMQFrame& frame){
+    ChannelId id = frame.getChannel();
     Channel* channel = channels[id];
-    // FIXME aconway 2007-01-26: Exception thrown here is hanging the
-    // client. Need to review use of exceptions.
     if (channel == 0)
         THROW_QPID_ERROR(
             PROTOCOL_ERROR+504,
             (boost::format("Invalid channel number %g") % id).str());
     try{
-        channel->handleBody(frame->getBody());
+        channel->getHandlers().in->handle(frame);
     }catch(const qpid::QpidError& e){
         channelException(
-            *channel, dynamic_cast<AMQMethodBody*>(frame->getBody().get()), e);
+            *channel, dynamic_cast<AMQMethodBody*>(frame.getBody().get()), e);
     }
 }
 
-void Connection::send(AMQFrame* frame) {
+void Connection::send(AMQFrame& frame) {
     out->send(frame);
 }
 
@@ -172,7 +169,8 @@
 }
 
 void Connection::idleOut(){
-    out->send(new AMQFrame(version, 0, new AMQHeartbeatBody()));
+    AMQFrame frame(version, 0, new AMQHeartbeatBody());
+    out->send(frame);
 }
 
 }} // namespace qpid::client

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connection.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connection.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connection.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connection.h Wed Jun 27 14:19:14 2007
@@ -159,8 +159,8 @@
 
 
     // TODO aconway 2007-01-26: can these be private?
-    void send(framing::AMQFrame*);
-    void received(framing::AMQFrame*);
+    void send(framing::AMQFrame&);
+    void received(framing::AMQFrame&);
     void idleOut();
     void idleIn();
     void shutdown();

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp Wed Jun 27 14:19:14 2007
@@ -22,6 +22,7 @@
 #include "qpid/log/Statement.h"
 #include "qpid/QpidError.h"
 #include "qpid/sys/Time.h"
+#include "qpid/framing/AMQFrame.h"
 #include "Connector.h"
 
 namespace qpid {
@@ -91,11 +92,9 @@
     return this; 
 }
 
-void Connector::send(AMQFrame* f){
-    std::auto_ptr<AMQFrame> frame(f);
-    AMQBody::shared_ptr body = frame->getBody();
-    writeBlock(frame.get());
-    QPID_LOG(trace, "SENT: " << *frame);
+void Connector::send(AMQFrame& frame){
+    writeBlock(&frame);
+    QPID_LOG(trace, "SENT: " << frame);
 }
 
 void Connector::writeBlock(AMQDataBlock* data){
@@ -185,7 +184,7 @@
 		AMQFrame frame(version);
 		while(frame.decode(inbuf)){
                     QPID_LOG(trace, "RECV: " << frame);
-		    input->received(&frame);
+		    input->received(frame);
 		}
                 //need to compact buffer to preserve any 'extra' data
                 inbuf.compact();

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h Wed Jun 27 14:19:14 2007
@@ -91,7 +91,7 @@
     virtual void setTimeoutHandler(sys::TimeoutHandler* handler);
     virtual void setShutdownHandler(sys::ShutdownHandler* handler);
     virtual framing::OutputHandler* getOutputHandler();
-    virtual void send(framing::AMQFrame* frame);
+    virtual void send(framing::AMQFrame& frame);
     virtual void setReadTimeout(uint16_t timeout);
     virtual void setWriteTimeout(uint16_t timeout);
 };

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.cpp Wed Jun 27 14:19:14 2007
@@ -19,6 +19,7 @@
 
 #include "ChannelAdapter.h"
 #include "AMQFrame.h"
+#include "FrameHandler.h"
 #include "qpid/Exception.h"
 
 using boost::format;
@@ -26,13 +27,21 @@
 namespace qpid {
 namespace framing {
 
-void ChannelAdapter::init(
-    ChannelId i, OutputHandler& o, ProtocolVersion v)
+/** Framehandler that feeds into the channel. */
+struct ChannelAdapter::ChannelAdapterHandler : public FrameHandler {
+    ChannelAdapterHandler(ChannelAdapter& channel_) : channel(channel_) {}
+    void handle(AMQFrame& frame) { channel.handleBody(frame.getBody()); }
+    ChannelAdapter& channel;
+};
+
+void ChannelAdapter::init(ChannelId i, OutputHandler& out, ProtocolVersion v)
 {
     assertChannelNotOpen();
     id = i;
-    out = &o;
     version = v;
+
+    handlers.in = make_shared_ptr(new ChannelAdapterHandler(*this));
+    handlers.out= make_shared_ptr(new OutputHandlerFrameHandler(out));
 }
 
 RequestId ChannelAdapter::send(
@@ -58,7 +67,8 @@
       }
         // No action required for other body types.
     }
-    out->send(new AMQFrame(getVersion(), getId(), body));
+    AMQFrame frame(getVersion(), getId(), body);
+    handlers.out->handle(frame);
     return requestId;
 }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/ChannelAdapter.h Wed Jun 27 14:19:14 2007
@@ -1,3 +1,6 @@
+
+
+
 #ifndef _ChannelAdapter_
 #define _ChannelAdapter_
 
@@ -28,40 +31,39 @@
 #include "Responder.h"
 #include "Correlator.h"
 #include "amqp_types.h"
+#include "FrameHandler.h"
 
 namespace qpid {
 namespace framing {
 
 class MethodContext;
 
-// FIXME aconway 2007-02-20: Rename as ChannelBase or just Channel.
-
 /**
  * Base class for client and broker channels.
  *
- * - receives frame bodies from the network.
- * - Updates request/response data.
- * - Dispatches requests with a MethodContext for responses.
+ * Provides in/out handler chains containing channel handlers.
+ * Chains may be modified by ChannelUpdaters registered with the broker.
+ * 
+ * The handlers provided by the ChannelAdapter update request/response data.
  *
- * send()
- * - Updates request/resposne ID data.
- * - Forwards frame to the peer.
+ * send() constructs a frame, updates request/resposne ID and forwards it
+ * to the out() chain.
  *
  * Thread safety: OBJECT UNSAFE. Instances must not be called
  * concurrently. AMQP defines channels to be serialized.
  */
-class ChannelAdapter : public BodyHandler {
+class ChannelAdapter : private BodyHandler {
   public:
     /**
      *@param output Processed frames are forwarded to this handler.
      */
-    ChannelAdapter(ChannelId id_=0, OutputHandler* out_=0,
-                   ProtocolVersion ver=ProtocolVersion())
-        : id(id_), out(out_), version(ver)  {}
+    ChannelAdapter() : id(0) {}
 
     /** Initialize the channel adapter. */
     void init(ChannelId, OutputHandler&, ProtocolVersion);
 
+    FrameHandler::Chains& getHandlers() { return handlers; }
+
     ChannelId getId() const { return id; }
     ProtocolVersion getVersion() const { return version; }
 
@@ -79,10 +81,6 @@
     /**@deprecated Use make_shared_ptr with the other send() override */
     RequestId send(AMQBody* body) { return send(AMQBody::shared_ptr(body)); }
 
-    void handleMethod(shared_ptr<AMQMethodBody>);
-    void handleRequest(shared_ptr<AMQRequestBody>);
-    void handleResponse(shared_ptr<AMQResponseBody>);
-
     virtual bool isOpen() const = 0;
     
   protected:
@@ -99,12 +97,19 @@
     RequestId getNextSendRequestId() { return requester.getNextId(); }
 
   private:
+    class ChannelAdapterHandler;
+  friend class ChannelAdapterHandler;
+    
+    void handleMethod(shared_ptr<AMQMethodBody>);
+    void handleRequest(shared_ptr<AMQRequestBody>);
+    void handleResponse(shared_ptr<AMQResponseBody>);
+
     ChannelId id;
-    OutputHandler* out;
     ProtocolVersion version;
     Requester requester;
     Responder responder;
     Correlator correlator;
+    FrameHandler::Chains handlers;
 };
 
 }}

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameHandler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameHandler.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/FrameHandler.h Wed Jun 27 14:19:14 2007
@@ -20,14 +20,14 @@
  * under the License.
  *
  */
-
 #include "Handler.h"
 
 namespace qpid {
 namespace framing {
+
 class AMQFrame;
 typedef Handler<AMQFrame&> FrameHandler;
-}}
 
 
+}}
 #endif  /*!QPID_FRAMING_FRAMEHANDLER_H*/

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Handler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Handler.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Handler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/Handler.h Wed Jun 27 14:19:14 2007
@@ -22,75 +22,29 @@
  *
  */
 #include "qpid/shared_ptr.h"
-#include <vector>
 #include <assert.h>
 
 namespace qpid {
 namespace framing {
 
-/** Handler for objects of type T. */
+/** Interface for handler for values of type T.
+ * Handlers can be linked into chains via the next pointer.
+ */
 template <class T> struct Handler {
-    typedef T Type;
-    typedef shared_ptr<Handler> Ptr;
+    typedef T ParamType;
+    typedef shared_ptr<Handler> Chain;
+
+    /** Handler chains for incoming and outgoing traffic. */
+    struct Chains {
+        Chain in;
+        Chain out;
+    };
 
     virtual ~Handler() {}
     virtual void handle(T) = 0;
-    virtual void link(Ptr next_) { next=next_; }
-  protected:
-    void nextHandler(T data) { if (next) next->handle(data); }
-  private:
-    Ptr next;
-};
-
-
-/** Factory interface that takes a context of type C */
-template <class T, class C> struct HandlerFactory {
-    virtual ~HandlerFactory() {}
-    typedef typename Handler<T>::Ptr Ptr;
-
-    /** Create a handler */
-    virtual Ptr create(C context) = 0;
-
-    /** Create a handler and link it to next */
-    Ptr create(C context, Ptr next) {
-        Ptr h=create(context);
-        h->link(next);
-    }
-};
-
-/** Factory implementation template */
-template <class FH, class C>
-struct HandlerFactoryImpl : public HandlerFactory<typename FH::Type, C> {
-    shared_ptr<Handler<typename FH::Type> > create(C context) {
-        return typename FH::Ptr(new FH(context));
-    }
+    Chain next;
 };
 
-/** A factory chain is a vector of handler factories used to create
- * handler chains. The chain does not own the factories.
- */
-template <class T, class C>
-struct HandlerFactoryChain : public std::vector<HandlerFactory<T,C>* > {
-    typedef typename Handler<T>::Ptr Ptr;
-    
-    /** Create a handler chain, return the first handler.
-     *@param context - passed to each factory.
-     */
-    Ptr create(C context) {
-        return this->create(context, this->begin());
-    }
-
-  private:
-    typedef typename std::vector<HandlerFactory<T,C>*>::iterator iterator;
-    Ptr create(C context, iterator i) {
-        if (i != this->end()) {
-            Ptr h=(*i)->create(context);
-            h->link(create(context, i+1));
-            return h;
-        }
-        return Ptr();
-    }
-};
 
 }}
 

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/InputHandler.h Wed Jun 27 14:19:14 2007
@@ -21,7 +21,7 @@
  *
  */
 
-#include "AMQFrame.h"
+#include "FrameHandler.h"
 #include <boost/noncopyable.hpp>
 
 namespace qpid {
@@ -30,7 +30,21 @@
 class InputHandler : private boost::noncopyable {
   public:
     virtual ~InputHandler() {}
-    virtual void received(AMQFrame* frame) = 0;
+    virtual void received(AMQFrame&) = 0;
+};
+
+/** FrameHandler that delegates to an InputHandler */
+struct InputHandlerFrameHandler : public FrameHandler {
+    InputHandlerFrameHandler(InputHandler& in_) : in(in_) {}
+    void handle(ParamType frame) { in.received(frame); }
+    InputHandler& in;
+};
+
+/** InputHandler that delegates to a FrameHandler */
+struct FrameHandlerInputHandler : public InputHandler {
+    FrameHandlerInputHandler(shared_ptr<FrameHandler> h) : handler(h) {}
+    void received(AMQFrame& frame) { handler->handle(frame); }
+    FrameHandler::Chain handler;
 };
 
 }}

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/OutputHandler.h Wed Jun 27 14:19:14 2007
@@ -22,16 +22,31 @@
  *
  */
 #include <boost/noncopyable.hpp>
+#include "FrameHandler.h"
 
 namespace qpid {
 namespace framing {
-class AMQFrame;
 
 class OutputHandler : private boost::noncopyable {
   public:
     virtual ~OutputHandler() {}
-    virtual void send(AMQFrame* frame) = 0;
+    virtual void send(AMQFrame&) = 0;
 };
+
+/** OutputHandler that delegates to a FrameHandler */
+struct FrameHandlerOutputHandler : public OutputHandler {
+    FrameHandlerOutputHandler(shared_ptr<FrameHandler> h) : handler(h) {}
+    void received(AMQFrame& frame) { handler->handle(frame); }
+    FrameHandler::Chain handler;
+};
+
+/** FrameHandler that delegates to an OutputHandler */
+struct OutputHandlerFrameHandler : public FrameHandler {
+    OutputHandlerFrameHandler(OutputHandler& out_) : out(out_) {}
+    void handle(ParamType frame) { out.send(frame); }
+    OutputHandler& out;
+};
+
 
 }}
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.cpp Wed Jun 27 14:19:14 2007
@@ -62,7 +62,7 @@
         try{
             while(frame.decode(in)){
                 QPID_LOG(debug, "RECV: " << frame);
-                handler->received(&frame);
+                handler->received(frame);
             }
         }catch(const std::exception& e){
             QPID_LOG(error, e.what());
@@ -94,14 +94,12 @@
             if(!framesToWrite.empty()){
                 out.clear();
                 bool encoded(false);
-                AMQFrame* frame = framesToWrite.front();
-                while(frame && out.available() >= frame->size()){
+                while(!framesToWrite.empty() && out.available() >= framesToWrite.front().size()){
+                    AMQFrame& frame = framesToWrite.front();
                     encoded = true;
-                    frame->encode(out);
-                    QPID_LOG(debug, "SENT: " << *frame);
-                    delete frame;
+                    frame.encode(out);
+                    QPID_LOG(debug, "SENT: " << frame);
                     framesToWrite.pop();
-                    frame = framesToWrite.empty() ? 0 : framesToWrite.front();
                 }
                 if(!encoded) THROW_QPID_ERROR(FRAMING_ERROR, "Could not write frame, too large for buffer.");
                 out.flip();
@@ -118,7 +116,7 @@
     }
 }
 
-void LFSessionContext::send(AMQFrame* frame){
+void LFSessionContext::send(AMQFrame& frame){
     Mutex::ScopedLock l(writeLock);
     if(!closing){
         framesToWrite.push(frame);

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/apr/LFSessionContext.h Wed Jun 27 14:19:14 2007
@@ -28,6 +28,7 @@
 #include <apr_time.h>
 
 #include "qpid/framing/AMQFrame.h"
+#include "qpid/framing/FrameHandler.h"
 #include "qpid/framing/Buffer.h"
 #include "qpid/sys/Monitor.h"
 #include "qpid/sys/Mutex.h"
@@ -55,7 +56,7 @@
 
     apr_pollfd_t fd;
 
-    std::queue<qpid::framing::AMQFrame*> framesToWrite;
+    std::queue<qpid::framing::AMQFrame> framesToWrite;
     qpid::sys::Mutex writeLock;
         
     bool processing;
@@ -66,7 +67,7 @@
                      LFProcessor* const processor, 
                      bool debug = false);
     virtual ~LFSessionContext();
-    virtual void send(qpid::framing::AMQFrame* frame);
+    virtual void send(framing::AMQFrame& frame);
     virtual void close();        
     void read();
     void write();

Modified: incubator/qpid/trunk/qpid/cpp/src/qpidd.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpidd.cpp?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpidd.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpidd.cpp Wed Jun 27 14:19:14 2007
@@ -57,7 +57,7 @@
 struct QpiddOptions : public qpid::Options {
     DaemonOptions daemon;
     Broker::Options broker;
-    log::Options log;
+    qpid::log::Options log;
     CommonOptions common;
     
     QpiddOptions() : qpid::Options("Options") {
@@ -126,7 +126,7 @@
 
     try {
         options.parse(argc, argv, options.common.config);
-        log::Logger::instance().configure(options.log, argv[0]);
+        qpid::log::Logger::instance().configure(options.log, argv[0]);
 
         // Options that just print information.
         if(options.common.help || options.common.version) {

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/BrokerChannelTest.cpp Wed Jun 27 14:19:14 2007
@@ -31,7 +31,7 @@
 #include "MockChannel.h"
 #include "qpid/broker/Connection.h"
 #include "qpid/framing/ProtocolInitiation.h"
-#include <boost/ptr_container/ptr_vector.hpp>
+#include <vector>
 
 using namespace boost;
 using namespace qpid::broker;
@@ -41,9 +41,9 @@
 using std::queue;
 
 struct MockHandler : ConnectionOutputHandler{
-    boost::ptr_vector<AMQFrame> frames; 
+    std::vector<AMQFrame> frames; 
 
-    void send(AMQFrame* frame){ frames.push_back(frame); }
+    void send(AMQFrame& frame){ frames.push_back(frame); }
 
     void close() {};
 };

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/InProcessBroker.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/InProcessBroker.h?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/InProcessBroker.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/InProcessBroker.h Wed Jun 27 14:19:14 2007
@@ -48,15 +48,15 @@
 
     /** A frame tagged with the sender */
     struct TaggedFrame {
-        TaggedFrame(Sender e, framing::AMQFrame* f) : frame(f), sender(e) {}
+        TaggedFrame(Sender e, framing::AMQFrame& f) : frame(f), sender(e) {}
         bool fromBroker() const { return sender == BROKER; }
         bool fromClient() const { return sender == CLIENT; }
 
         template <class MethodType>
         MethodType* asMethod() {
-            return dynamic_cast<MethodType*>(frame->getBody().get());
+            return dynamic_cast<MethodType*>(frame.getBody().get());
         }
-        shared_ptr<framing::AMQFrame> frame;
+        framing::AMQFrame frame;
         Sender sender;
     };
     
@@ -85,7 +85,7 @@
     }
 
     /** Called by client to send a frame */
-    void send(framing::AMQFrame* frame) {
+    void send(framing::AMQFrame& frame) {
         clientOut.send(frame);
     }
 
@@ -100,7 +100,7 @@
             framing::InputHandler* ih=0
         ) : sender(sender_), conversation(conversation_), in(ih) {}
 
-        void send(framing::AMQFrame* frame) {
+        void send(framing::AMQFrame& frame) {
             conversation.push_back(TaggedFrame(sender, frame));
             in->received(frame);
         }
@@ -122,7 +122,7 @@
 std::ostream& operator<<(
     std::ostream& out, const InProcessBroker::TaggedFrame& tf)
 {
-    return out << (tf.fromBroker()? "BROKER: ":"CLIENT: ") << *tf.frame;
+    return out << (tf.fromBroker()? "BROKER: ":"CLIENT: ") << tf.frame;
 }
 
 std::ostream& operator<<(

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/Makefile.am Wed Jun 27 14:19:14 2007
@@ -25,10 +25,6 @@
 Url_SOURCES=Url.cpp test_tools.h
 Url_LDADD=-lboost_unit_test_framework $(lib_common) 
 
-unit_progs+=Handler
-Handler_SOURCES=Handler.cpp test_tools.h
-Handler_LDADD=-lboost_unit_test_framework $(lib_common) 
-
 include cluster.mk
 
 # NB: CppUnit test libraries below will be migrated to boost test programs.

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?view=diff&rev=551336&r1=551335&r2=551336
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/MockChannel.h Wed Jun 27 14:19:14 2007
@@ -25,12 +25,12 @@
 #include "qpid/framing/AMQFrame.h"
 #include "qpid/framing/BasicGetBody.h"
 #include <boost/shared_ptr.hpp>
-#include <boost/ptr_container/ptr_vector.hpp>
+#include <vector>
 
 /** Mock output handler to collect frames */
 struct MockOutputHandler : public qpid::framing::OutputHandler {
-    boost::ptr_vector<qpid::framing::AMQFrame> frames;
-    void send(qpid::framing::AMQFrame* frame){ frames.push_back(frame); }
+    std::vector<qpid::framing::AMQFrame> frames;
+    void send(qpid::framing::AMQFrame& frame){ frames.push_back(frame); }
 };
 
 /**