You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by gs...@apache.org on 2007/09/21 17:13:54 UTC

svn commit: r578167 - in /incubator/qpid/trunk/qpid: cpp/ cpp/src/ cpp/src/qpid/broker/ cpp/src/tests/ python/tests_0-10/ specs/

Author: gsim
Date: Fri Sep 21 08:13:52 2007
New Revision: 578167

URL: http://svn.apache.org/viewvc?rev=578167&view=rev
Log:
Moved c++ over to using the same preview file for 0-10 work as java.
Removed all channel class related code from broker as a result.
Did the same for some python tests I missed earlier.
Renamed ChannelAdapter to ChannelHandler.


Added:
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.cpp
      - copied, changed from r577590, incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.h
      - copied, changed from r577590, incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.h
Removed:
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.h
    incubator/qpid/trunk/qpid/specs/amqp-transitional.0-10.xml
Modified:
    incubator/qpid/trunk/qpid/cpp/configure.ac
    incubator/qpid/trunk/qpid/cpp/src/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.h
    incubator/qpid/trunk/qpid/cpp/src/tests/python_tests
    incubator/qpid/trunk/qpid/python/tests_0-10/alternate-exchange.py
    incubator/qpid/trunk/qpid/python/tests_0-10/dtx.py
    incubator/qpid/trunk/qpid/python/tests_0-10/message.py
    incubator/qpid/trunk/qpid/specs/amqp.0-10-preview.xml

Modified: incubator/qpid/trunk/qpid/cpp/configure.ac
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/configure.ac?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/configure.ac (original)
+++ incubator/qpid/trunk/qpid/cpp/configure.ac Fri Sep 21 08:13:52 2007
@@ -120,7 +120,7 @@
 test -z "$RUBY" && AC_MSG_ERROR([Missing ruby installation (try "yum install ruby").])
 
 specdir=`pwd`/$srcdir/../specs  
-AMQP_XML=$specdir/amqp-transitional.0-10.xml
+AMQP_XML=$specdir/amqp.0-10-preview.xml
 AC_SUBST(AMQP_XML)
 ls $AMQP_XML >/dev/null 2>&1 || generate=no
 

Modified: incubator/qpid/trunk/qpid/cpp/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/Makefile.am?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/Makefile.am Fri Sep 21 08:13:52 2007
@@ -157,7 +157,7 @@
   qpid/broker/BrokerExchange.cpp \
   qpid/broker/BrokerQueue.cpp \
   qpid/broker/Connection.cpp \
-  qpid/broker/ConnectionAdapter.cpp \
+  qpid/broker/ConnectionHandler.cpp \
   qpid/broker/ConnectionFactory.cpp \
   qpid/broker/Daemon.cpp \
   qpid/broker/DeliverableMessage.cpp \
@@ -268,7 +268,7 @@
   qpid/broker/BrokerAdapter.h \
   qpid/broker/BrokerSingleton.h \
   qpid/broker/Connection.h \
-  qpid/broker/ConnectionAdapter.h \
+  qpid/broker/ConnectionHandler.h \
   qpid/broker/ConnectionFactory.h \
   qpid/broker/ConnectionToken.h \
   qpid/broker/Daemon.h \

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?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h Fri Sep 21 08:13:52 2007
@@ -36,7 +36,7 @@
 #include "Broker.h"
 #include "qpid/Exception.h"
 #include "Session.h"
-#include "ConnectionAdapter.h"
+#include "ConnectionHandler.h"
 #include "SessionHandler.h"
 
 #include <boost/optional.hpp>
@@ -95,7 +95,7 @@
     uint16_t heartbeat;
     framing::AMQP_ClientProxy::Connection* client;
     uint64_t stagingThreshold;
-    ConnectionAdapter adapter;
+    ConnectionHandler adapter;
 };
 
 }}

Copied: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.cpp (from r577590, incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.cpp)
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.cpp?p2=incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.cpp&p1=incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.cpp&r1=577590&r2=578167&rev=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.cpp Fri Sep 21 08:13:52 2007
@@ -20,7 +20,7 @@
  *
  */
 
-#include "ConnectionAdapter.h"
+#include "ConnectionHandler.h"
 #include "Connection.h"
 #include "qpid/framing/ConnectionStartBody.h"
 
@@ -28,34 +28,24 @@
 using namespace qpid::broker;
 using namespace qpid::framing;
 
-void ConnectionAdapter::init(const framing::ProtocolInitiation& header) {
+void ConnectionHandler::init(const framing::ProtocolInitiation& header) {
     FieldTable properties;
     string mechanisms("PLAIN");
     string locales("en_US");
     handler->client.start(header.getMajor(), header.getMinor(), properties, mechanisms, locales);
 }
 
-void ConnectionAdapter::close(ReplyCode code, const string& text, ClassId classId, MethodId methodId)
+void ConnectionHandler::close(ReplyCode code, const string& text, ClassId classId, MethodId methodId)
 {
     handler->client.close(code, text, classId, methodId);
 }
 
-
-framing::AMQP_ServerOperations::ConnectionHandler* ConnectionAdapter::getConnectionHandler() 
-{ 
-    return handler.get(); 
-}
-
-framing::ProtocolVersion ConnectionAdapter::getVersion() const 
-{ 
-    return handler->connection.getVersion(); 
-}
-
-void ConnectionAdapter::handle(framing::AMQFrame& frame)
+void ConnectionHandler::handle(framing::AMQFrame& frame)
 {
     AMQMethodBody* method=frame.getBody()->getMethod();
     try{
-        method->invoke(*this);
+        if (!method->invoke(handler.get()))
+            throw ConnectionException(503, "Class can't be accessed over channel 0");
     }catch(ConnectionException& e){
         handler->client.close(e.code, e.toString(), method->amqpClassId(), method->amqpMethodId());
     }catch(std::exception& e){
@@ -63,27 +53,27 @@
     }
 }
 
-ConnectionAdapter::ConnectionAdapter(Connection& connection)  : handler(new Handler(connection)) {}
+ConnectionHandler::ConnectionHandler(Connection& connection)  : handler(new Handler(connection)) {}
 
-ConnectionAdapter::Handler:: Handler(Connection& c) : client(c.getOutput()), connection(c) {}
+ConnectionHandler::Handler:: Handler(Connection& c) : client(c.getOutput()), connection(c) {}
 
-void ConnectionAdapter::Handler::startOk(const FieldTable& /*clientProperties*/,
+void ConnectionHandler::Handler::startOk(const FieldTable& /*clientProperties*/,
     const string& /*mechanism*/, 
     const string& /*response*/, const string& /*locale*/)
 {
     client.tune(framing::CHANNEL_MAX, connection.getFrameMax(), connection.getHeartbeat());
 }
         
-void ConnectionAdapter::Handler::secureOk(const string& /*response*/){}
+void ConnectionHandler::Handler::secureOk(const string& /*response*/){}
         
-void ConnectionAdapter::Handler::tuneOk(uint16_t /*channelmax*/,
+void ConnectionHandler::Handler::tuneOk(uint16_t /*channelmax*/,
     uint32_t framemax, uint16_t heartbeat)
 {
     connection.setFrameMax(framemax);
     connection.setHeartbeat(heartbeat);
 }
         
-void ConnectionAdapter::Handler::open(const string& /*virtualHost*/,
+void ConnectionHandler::Handler::open(const string& /*virtualHost*/,
     const string& /*capabilities*/, bool /*insist*/)
 {
     string knownhosts;
@@ -91,13 +81,13 @@
 }
 
         
-void ConnectionAdapter::Handler::close(uint16_t /*replyCode*/, const string& /*replyText*/, 
+void ConnectionHandler::Handler::close(uint16_t /*replyCode*/, const string& /*replyText*/, 
     uint16_t /*classId*/, uint16_t /*methodId*/)
 {
     client.closeOk();
     connection.getOutput().close();
 } 
         
-void ConnectionAdapter::Handler::closeOk(){
+void ConnectionHandler::Handler::closeOk(){
     connection.getOutput().close();
 } 

Copied: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.h (from r577590, incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.h)
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.h?p2=incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.h&p1=incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.h&r1=577590&r2=578167&rev=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionAdapter.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionHandler.h Fri Sep 21 08:13:52 2007
@@ -37,7 +37,7 @@
 class Connection;
 
 // TODO aconway 2007-09-18: Rename to ConnectionHandler
-class ConnectionAdapter : public framing::FrameHandler, public framing::AMQP_ServerOperations
+class ConnectionHandler : public framing::FrameHandler
 {
     struct Handler : public framing::AMQP_ServerOperations::ConnectionHandler
     {
@@ -58,29 +58,10 @@
     };
     std::auto_ptr<Handler> handler;
   public:
-    ConnectionAdapter(Connection& connection);
+    ConnectionHandler(Connection& connection);
     void init(const framing::ProtocolInitiation& header);
     void close(framing::ReplyCode code, const std::string& text, framing::ClassId classId, framing::MethodId methodId);
     void handle(framing::AMQFrame& frame);
-
-    //AMQP_ServerOperations:
-    ConnectionHandler* getConnectionHandler();
-    ChannelHandler* getChannelHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    BasicHandler* getBasicHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    ExchangeHandler* getExchangeHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    BindingHandler* getBindingHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    QueueHandler* getQueueHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    TxHandler* getTxHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    MessageHandler* getMessageHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    AccessHandler* getAccessHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    FileHandler* getFileHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    StreamHandler* getStreamHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    TunnelHandler* getTunnelHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    DtxCoordinationHandler* getDtxCoordinationHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    DtxDemarcationHandler* getDtxDemarcationHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    ExecutionHandler* getExecutionHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    SessionHandler* getSessionHandler() { throw ConnectionException(503, "Class can't be accessed over channel 0"); }
-    framing::ProtocolVersion getVersion() const;
 };
 
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.cpp?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SemanticHandler.cpp Fri Sep 21 08:13:52 2007
@@ -28,7 +28,6 @@
 #include "Session.h"
 #include "qpid/framing/ExecutionCompleteBody.h"
 #include "qpid/framing/ExecutionResultBody.h"
-#include "qpid/framing/ChannelOpenBody.h"
 #include "qpid/framing/InvocationVisitor.h"
 
 #include <boost/format.hpp>

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.cpp Fri Sep 21 08:13:52 2007
@@ -33,8 +33,7 @@
 SessionHandler::SessionHandler(Connection& c, ChannelId ch)
     : InOutHandler(0, &c.getOutput()),
       connection(c), channel(ch), proxy(out),
-      ignoring(false), channelHandler(*this),
-      useChannelClose(false) {}
+      ignoring(false) {}
 
 SessionHandler::~SessionHandler() {}
 
@@ -52,7 +51,7 @@
     // 
     AMQMethodBody* m=f.getMethod();
     try {
-        if (m && (m->invoke(this) || m->invoke(&channelHandler)))
+        if (m && m->invoke(this))
             return;
         else if (session)
             session->in(f);
@@ -62,12 +61,7 @@
     } catch(const ChannelException& e) {
         ignoring=true;          // Ignore trailing frames sent by client.
         session.reset();
-        // FIXME aconway 2007-09-19: Dual-mode hack.
-        if (useChannelClose)
-            getProxy().getChannel().close(
-                e.code, e.toString(), classId(m), methodId(m));
-        else
-            getProxy().getSession().closed(e.code, e.toString());
+        getProxy().getSession().closed(e.code, e.toString());
     }catch(const ConnectionException& e){
         connection.close(e.code, e.what(), classId(m), methodId(m));
     }catch(const std::exception& e){
@@ -96,51 +90,6 @@
             QPID_MSG(""<<method<<" failed: "
                      << channel << " already open on channel "
                      << getChannel()));
-}
-
-void SessionHandler::ChannelMethods::open(const string& /*outOfBand*/){
-    parent.useChannelClose=true;
-    parent.assertClosed("open");
-    parent.session.reset(new Session(parent, 0));
-    parent.getProxy().getChannel().openOk();
-} 
-
-// FIXME aconway 2007-08-31: flow is no longer in the spec.
-void SessionHandler::ChannelMethods::flow(bool active){
-    parent.session->flow(active);
-    parent.getProxy().getChannel().flowOk(active);
-}
-
-void SessionHandler::ChannelMethods::flowOk(bool /*active*/){}
-        
-void SessionHandler::ChannelMethods::close(uint16_t replyCode,
-                           const string& replyText,
-                           uint16_t classId, uint16_t methodId)
-{
-    // FIXME aconway 2007-08-31: Extend constants.h to map codes & ids
-    // to text names.
-    QPID_LOG(warning, "Received channel.close("<<replyCode<<","
-             <<replyText << ","
-             << "classid=" <<classId<< ","
-             << "methodid=" <<methodId);
-    parent.ignoring=false;
-    parent.getProxy().getChannel().closeOk();
-    // FIXME aconway 2007-08-31: sould reset session BEFORE
-    // sending closeOK to avoid races. SessionHandler
-    // needs its own private proxy, see getProxy() above.
-    parent.session.reset();
-    // No need to remove from connection map, will be re-used
-    // if channel is re-opened.
-} 
-        
-void SessionHandler::ChannelMethods::closeOk(){
-    parent.ignoring=false;
-}
-
-void SessionHandler::ChannelMethods::ok() 
-{
-    //no specific action required, generic response handling should be
-    //sufficient
 }
 
 void  SessionHandler::open(uint32_t detachedLifetime) {

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.h?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionHandler.h Fri Sep 21 08:13:52 2007
@@ -64,25 +64,6 @@
     void handleOut(framing::AMQFrame&);
     
   private:
-    // FIXME aconway 2007-08-31: Drop channel.
-    struct ChannelMethods : public framing::AMQP_ServerOperations::ChannelHandler {
-        SessionHandler& parent;
-
-        ChannelMethods(SessionHandler& p) : parent(p) {}
-        void open(const std::string& outOfBand); 
-        void flow(bool active); 
-        void flowOk(bool active); 
-        void ok(  );
-        void ping(  );
-        void pong(  );
-        void resume( const std::string& channelId );
-        void close(uint16_t replyCode,
-                   const std::string& replyText,
-                   uint16_t classId, uint16_t methodId); 
-        void closeOk(); 
-    };
-  friend class ChannelMethods;
-
     /// Session methods
     void open(uint32_t detachedLifetime);
     void flow(bool active);
@@ -105,8 +86,6 @@
     framing::AMQP_ClientProxy proxy;
     shared_ptr<Session> session;
     bool ignoring;
-    ChannelMethods channelHandler;
-    bool useChannelClose;       // FIXME aconway 2007-09-19: remove with channel.
 };
 
 }} // namespace qpid::broker

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/python_tests
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/python_tests?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/python_tests (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/python_tests Fri Sep 21 08:13:52 2007
@@ -1,7 +1,7 @@
 #!/bin/sh
 # Run the python tests.
 if test -d ../../../python ;  then
-    cd ../../../python && ./run-tests -v -s ../specs/amqp-transitional.0-10.xml -I cpp_failing_0-10.txt -b localhost:$QPID_PORT $PYTHON_TESTS
+    cd ../../../python && ./run-tests -v -s ../specs/amqp.0-10-preview.xml -I cpp_failing_0-10.txt -b localhost:$QPID_PORT $PYTHON_TESTS
 else
     echo Warning: python tests not found.
 fi

Modified: incubator/qpid/trunk/qpid/python/tests_0-10/alternate-exchange.py
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/python/tests_0-10/alternate-exchange.py?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/python/tests_0-10/alternate-exchange.py (original)
+++ incubator/qpid/trunk/qpid/python/tests_0-10/alternate-exchange.py Fri Sep 21 08:13:52 2007
@@ -136,9 +136,9 @@
             #cleanup:
             other = self.connect()
             channel = other.channel(1)
-            channel.channel_open()
+            channel.session_open()
             channel.exchange_delete(exchange="alternate")
-            channel.channel_close(200, "ok")
+            channel.session_close()
             other.close()
             
             self.assertConnectionException(530, e.args[0])            
@@ -162,10 +162,10 @@
             #cleanup:
             other = self.connect()
             channel = other.channel(1)
-            channel.channel_open()
+            channel.session_open()
             channel.exchange_delete(exchange="e")
             channel.exchange_delete(exchange="alternate")
-            channel.channel_close(200, "ok")
+            channel.session_close()
             other.close()
 
             self.assertConnectionException(530, e.args[0])

Modified: incubator/qpid/trunk/qpid/python/tests_0-10/dtx.py
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/python/tests_0-10/dtx.py?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/python/tests_0-10/dtx.py (original)
+++ incubator/qpid/trunk/qpid/python/tests_0-10/dtx.py Fri Sep 21 08:13:52 2007
@@ -461,8 +461,11 @@
 
         channel2.dtx_demarcation_select()
         channel2.dtx_demarcation_start(xid=tx)
-        channel2.message_get(queue="dummy", destination="dummy")
+        channel2.message_subscribe(queue="dummy", destination="dummy", confirm_mode=1)
+        channel2.message_flow(destination="dummy", unit=0, value=1)
+        channel2.message_flow(destination="dummy", unit=1, value=0xFFFFFFFF)
         self.client.queue("dummy").get(timeout=1).complete()
+        channel2.message_cancel(destination="dummy")
         channel2.message_transfer(content=Content(properties={'routing_key':"dummy"}, body="whatever"))
         channel2.session_close()
 
@@ -589,8 +592,11 @@
 
     def swap(self, channel, src, dest):
         #consume from src:
-        channel.message_get(destination="temp-swap", queue=src)
+        channel.message_subscribe(destination="temp-swap", queue=src, confirm_mode=1)
+        channel.message_flow(destination="temp-swap", unit=0, value=1)
+        channel.message_flow(destination="temp-swap", unit=1, value=0xFFFFFFFF)
         msg = self.client.queue("temp-swap").get(timeout=1)
+        channel.message_cancel(destination="temp-swap")
         msg.complete();        
 
         #re-publish to dest

Modified: incubator/qpid/trunk/qpid/python/tests_0-10/message.py
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/python/tests_0-10/message.py?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/python/tests_0-10/message.py (original)
+++ incubator/qpid/trunk/qpid/python/tests_0-10/message.py Fri Sep 21 08:13:52 2007
@@ -70,7 +70,7 @@
 
         #open new channel and cleanup last consumer:    
         channel = self.client.channel(2)
-        channel.channel_open()
+        channel.session_open()
 
         #check that an exclusive consumer cannot be created if a consumer already exists:
         self.subscribe(channel, destination="first", queue="test-queue-2")
@@ -93,7 +93,7 @@
             self.assertChannelException(404, e.args[0])
 
         channel = self.client.channel(2)
-        channel.channel_open()
+        channel.session_open()
         try:
             #queue not specified and none previously declared for channel:
             self.subscribe(channel, queue="", destination="")

Modified: incubator/qpid/trunk/qpid/specs/amqp.0-10-preview.xml
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/specs/amqp.0-10-preview.xml?rev=578167&r1=578166&r2=578167&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/specs/amqp.0-10-preview.xml (original)
+++ incubator/qpid/trunk/qpid/specs/amqp.0-10-preview.xml Fri Sep 21 08:13:52 2007
@@ -2068,7 +2068,9 @@
       <chassis name="client" implement="MUST" />
       <chassis name="server" implement="MUST" />
 
+      <!--
       <response name="closed" />
+      -->
     </method>
 
     <!-- - Method: session.closed  - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -->
@@ -6926,7 +6928,7 @@
 
     <!-- - Method: message.empty - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -->
 
-    <method name="empty" index="510" label="empty queue">
+    <method name="empty" index="200" label="empty queue">
       <doc>
         Signals that a queue does not contain any messages; usually sent in response to the get
         method.
@@ -6938,7 +6940,7 @@
 
     <!-- - Method: message.offset  - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -->
 
-    <method name="offset" index="530" label="return an offset">
+    <method name="offset" index="210" label="return an offset">
       <doc>
         Returns the data offset into a reference body; usually sent in response to resume method.
       </doc>