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 2009/11/09 16:30:56 UTC

svn commit: r834108 - in /qpid/trunk/qpid/cpp: src/qpid/broker/ src/qpid/client/ src/qpid/cluster/ src/qpid/sys/ src/qpid/sys/ssl/ xml/

Author: gsim
Date: Mon Nov  9 15:30:18 2009
New Revision: 834108

URL: http://svn.apache.org/viewvc?rev=834108&view=rev
Log:
QPID-1899: Applied patch from Ken Giusti to tie in SASL enctryption to the handling of the --require-encrypted option

Modified:
    qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h
    qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h
    qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp
    qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h
    qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h
    qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/Connector.h
    qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h
    qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp
    qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp
    qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
    qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp
    qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h
    qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp
    qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h
    qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h
    qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h
    qpid/trunk/qpid/cpp/xml/cluster.xml

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp Mon Nov  9 15:30:18 2009
@@ -71,8 +71,9 @@
     }
 };
 
-Connection::Connection(ConnectionOutputHandler* out_, Broker& broker_, const std::string& mgmtId_, bool isLink_, uint64_t objectId) :
+Connection::Connection(ConnectionOutputHandler* out_, Broker& broker_, const std::string& mgmtId_, unsigned int ssf, bool isLink_, uint64_t objectId) :
     ConnectionState(out_, broker_),
+    ssf(ssf),
     adapter(*this, isLink_),
     isLink(isLink_),
     mgmtClosing(false),

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h Mon Nov  9 15:30:18 2009
@@ -78,7 +78,8 @@
         virtual void connectionError(const std::string&) = 0;
     };
 
-    Connection(sys::ConnectionOutputHandler* out, Broker& broker, const std::string& mgmtId, bool isLink = false, uint64_t objectId = 0);
+    Connection(sys::ConnectionOutputHandler* out, Broker& broker, const std::string& mgmtId, unsigned int ssf,
+               bool isLink = false, uint64_t objectId = 0);
     ~Connection ();
 
     /** Get the SessionHandler for channel. Create if it does not already exist */
@@ -138,11 +139,14 @@
     // Used by cluster to update connection status
     sys::AggregateOutput& getOutputTasks() { return outputTasks; }
 
+    unsigned int getSSF() { return ssf; }
+
   private:
     typedef boost::ptr_map<framing::ChannelId, SessionHandler> ChannelMap;
     typedef std::vector<Queue::shared_ptr>::iterator queue_iterator;
 
     ChannelMap channels;
+    unsigned int ssf;
     ConnectionHandler adapter;
     const bool isLink;
     bool mgmtClosing;

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp Mon Nov  9 15:30:18 2009
@@ -35,7 +35,8 @@
 ConnectionFactory::~ConnectionFactory() {}
 
 sys::ConnectionCodec*
-ConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id) {
+ConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
+                          unsigned int ) {
     if (v == ProtocolVersion(0, 10)) {
         ConnectionPtr c(new amqp_0_10::Connection(out, id, false));
         c->setInputHandler(InputPtr(new broker::Connection(c.get(), broker, id, false)));
@@ -45,7 +46,8 @@
 }
 
 sys::ConnectionCodec*
-ConnectionFactory::create(sys::OutputControl& out, const std::string& id) {
+ConnectionFactory::create(sys::OutputControl& out, const std::string& id,
+                          unsigned int) {
     // used to create connections from one broker to another
     ConnectionPtr c(new amqp_0_10::Connection(out, id, true));
     c->setInputHandler(InputPtr(new broker::Connection(c.get(), broker, id, true)));

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h Mon Nov  9 15:30:18 2009
@@ -35,10 +35,12 @@
     virtual ~ConnectionFactory();
 
     sys::ConnectionCodec*
-    create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id);
+    create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id,
+           unsigned int conn_ssf);
 
     sys::ConnectionCodec*
-    create(sys::OutputControl&, const std::string& id);
+    create(sys::OutputControl&, const std::string& id,
+           unsigned int conn_ssf);
 
   private:
     Broker& broker;

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp Mon Nov  9 15:30:18 2009
@@ -48,8 +48,9 @@
     Connection& connection;
     framing::AMQP_ClientProxy::Connection client;
     std::string realm;
+    const bool encrypt;
 public:
-    NullAuthenticator(Connection& connection, bool dummy=false/*dummy arg to match CyrusAuthenticator*/);
+    NullAuthenticator(Connection& connection, bool encrypt);
     ~NullAuthenticator();
     void getMechanisms(framing::Array& mechanisms);
     void start(const std::string& mechanism, const std::string& response);
@@ -130,12 +131,12 @@
     } else {
         QPID_LOG(debug, "SASL: No Authentication Performed");
         needWarning = false;
-        return std::auto_ptr<SaslAuthenticator>(new NullAuthenticator(c));
+        return std::auto_ptr<SaslAuthenticator>(new NullAuthenticator(c, c.getBroker().getOptions().requireEncrypted));
     }
 }
 
-  NullAuthenticator::NullAuthenticator(Connection& c, bool /*dummy*/) : connection(c), client(c.getOutput()), 
-                                                      realm(c.getBroker().getOptions().realm) {}
+NullAuthenticator::NullAuthenticator(Connection& c, bool e) : connection(c), client(c.getOutput()), 
+                                                              realm(c.getBroker().getOptions().realm), encrypt(e) {}
 NullAuthenticator::~NullAuthenticator() {}
 
 void NullAuthenticator::getMechanisms(Array& mechanisms)
@@ -146,6 +147,10 @@
 
 void NullAuthenticator::start(const string& mechanism, const string& response)
 {
+    if (encrypt) {
+        QPID_LOG(error, "Rejected un-encrypted connection.");
+        throw ConnectionForcedException("Connection must be encrypted.");
+    }
     if (mechanism == "PLAIN") { // Old behavior
         if (response.size() > 0) {
             string uid;
@@ -227,10 +232,24 @@
     //TODO: should the actual SSF values be configurable here?
     secprops.min_ssf = encrypt ? 10: 0;
     secprops.max_ssf = 256;
-    secprops.maxbufsize = 65535;
 
-    QPID_LOG(debug, "min_ssf: " << secprops.min_ssf << ", max_ssf: " << secprops.max_ssf);
-    
+    // If the transport provides encryption, notify the SASL library of
+    // the key length and set the ssf range to prevent double encryption.
+    sasl_ssf_t external_ssf = (sasl_ssf_t) connection.getSSF();
+    if (external_ssf) {
+        int result = sasl_setprop(sasl_conn, SASL_SSF_EXTERNAL, &external_ssf);
+        if (result != SASL_OK) {
+            throw framing::InternalErrorException(QPID_MSG("SASL error: unable to set external SSF: " << result));
+        }
+
+        secprops.max_ssf = secprops.min_ssf = 0;
+    }
+
+    QPID_LOG(debug, "min_ssf: " << secprops.min_ssf <<
+             ", max_ssf: " << secprops.max_ssf <<
+             ", external_ssf: " << external_ssf );
+
+    secprops.maxbufsize = 65535;
     secprops.property_names = 0;
     secprops.property_values = 0;
     secprops.security_flags = 0; /* or SASL_SEC_NOANONYMOUS etc as appropriate */

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp Mon Nov  9 15:30:18 2009
@@ -36,11 +36,12 @@
 SecureConnectionFactory::SecureConnectionFactory(Broker& b) : broker(b) {}
 
 sys::ConnectionCodec*
-SecureConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id) {
+SecureConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
+                                unsigned int conn_ssf ) {
     if (v == ProtocolVersion(0, 10)) {
         SecureConnectionPtr sc(new SecureConnection());
         CodecPtr c(new amqp_0_10::Connection(out, id, false));
-        ConnectionPtr i(new broker::Connection(c.get(), broker, id, false));
+        ConnectionPtr i(new broker::Connection(c.get(), broker, id, conn_ssf, false));
         i->setSecureConnection(sc.get());
         c->setInputHandler(InputPtr(i.release()));
         sc->setCodec(std::auto_ptr<sys::ConnectionCodec>(c));
@@ -50,11 +51,12 @@
 }
 
 sys::ConnectionCodec*
-SecureConnectionFactory::create(sys::OutputControl& out, const std::string& id) {
+SecureConnectionFactory::create(sys::OutputControl& out, const std::string& id,
+                                unsigned int conn_ssf) {
     // used to create connections from one broker to another
     SecureConnectionPtr sc(new SecureConnection());
     CodecPtr c(new amqp_0_10::Connection(out, id, true));
-    ConnectionPtr i(new broker::Connection(c.get(), broker, id, true));
+    ConnectionPtr i(new broker::Connection(c.get(), broker, id, conn_ssf, true ));
     i->setSecureConnection(sc.get());
     c->setInputHandler(InputPtr(i.release()));
     sc->setCodec(std::auto_ptr<sys::ConnectionCodec>(c));

Modified: qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h Mon Nov  9 15:30:18 2009
@@ -33,10 +33,12 @@
     SecureConnectionFactory(Broker& b);            
 
     sys::ConnectionCodec*
-    create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id);
+    create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id,
+           unsigned int conn_ssf);
 
     sys::ConnectionCodec*
-    create(sys::OutputControl&, const std::string& id);
+    create(sys::OutputControl&, const std::string& id,
+           unsigned int conn_ssf);
 
   private:
     Broker& broker;

Modified: qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp Mon Nov  9 15:30:18 2009
@@ -212,7 +212,8 @@
     }
 
     if (sasl.get()) {
-        string response = sasl->start(mechanism.empty() ? mechlist : mechanism);
+        string response = sasl->start(mechanism.empty() ? mechlist : mechanism,
+                                      getSSF ? getSSF() : 0);
         proxy.startOk(properties, sasl->getMechanism(), response, locale);
     } else {
         //TODO: verify that desired mechanism and locale are supported

Modified: qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h Mon Nov  9 15:30:18 2009
@@ -94,7 +94,8 @@
 public:
     using InputHandler::handle;
     typedef boost::function<void()> CloseListener;    
-    typedef boost::function<void(uint16_t, const std::string&)> ErrorListener;    
+    typedef boost::function<void(uint16_t, const std::string&)> ErrorListener;
+    typedef boost::function<unsigned int()> GetConnSSF;
 
     ConnectionHandler(const ConnectionSettings&, framing::ProtocolVersion&);
 
@@ -122,6 +123,7 @@
 
     static framing::connection::CloseCode convert(uint16_t replyCode);
     const std::string& getUserId() const { return operUserId; }
+    GetConnSSF  getSSF;     /** query the connection for its security strength factor */
 };
 
 }}

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=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp Mon Nov  9 15:30:18 2009
@@ -79,6 +79,7 @@
                                   CLOSE_CODE_NORMAL, std::string());
     //only set error handler once  open
     handler.onError = boost::bind(&ConnectionImpl::closed, this, _1, _2);
+    handler.getSSF = boost::bind(&Connector::getSSF, boost::ref(connector));
 }
 
 const uint16_t ConnectionImpl::NEXT_CHANNEL = std::numeric_limits<uint16_t>::max();

Modified: qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/Connector.cpp Mon Nov  9 15:30:18 2009
@@ -148,6 +148,7 @@
     TCPConnector(framing::ProtocolVersion pVersion,
               const ConnectionSettings&, 
               ConnectionImpl*);
+    unsigned int getSSF() { return 0; }
 };
 
 // Static constructor which registers connector here

Modified: qpid/trunk/qpid/cpp/src/qpid/client/Connector.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/Connector.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/Connector.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/Connector.h Mon Nov  9 15:30:18 2009
@@ -74,6 +74,8 @@
 
     virtual void activateSecurityLayer(std::auto_ptr<qpid::sys::SecurityLayer>);
 
+    virtual unsigned int getSSF() = 0;
+
 };
 
 }}

Modified: qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp Mon Nov  9 15:30:18 2009
@@ -122,6 +122,7 @@
     RdmaConnector(framing::ProtocolVersion pVersion,
               const ConnectionSettings&, 
               ConnectionImpl*);
+    unsigned int getSSF() { return 0; }
 };
 
 // Static constructor which registers connector here

Modified: qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h Mon Nov  9 15:30:18 2009
@@ -42,7 +42,7 @@
 class Sasl
 {
   public:
-    virtual std::string start(const std::string& mechanisms) = 0;
+    virtual std::string start(const std::string& mechanisms, unsigned int ssf) = 0;
     virtual std::string step(const std::string& challenge) = 0;
     virtual std::string getMechanism() = 0;
     virtual std::string getUserId() = 0;

Modified: qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp Mon Nov  9 15:30:18 2009
@@ -80,7 +80,7 @@
   public:
     CyrusSasl(const ConnectionSettings&);
     ~CyrusSasl();
-    std::string start(const std::string& mechanisms);
+    std::string start(const std::string& mechanisms, unsigned int ssf);
     std::string step(const std::string& challenge);
     std::string getMechanism();
     std::string getUserId();
@@ -176,7 +176,7 @@
     const std::string SSL("ssl");
 }
 
-std::string CyrusSasl::start(const std::string& mechanisms)
+std::string CyrusSasl::start(const std::string& mechanisms, unsigned int ssf)
 {
     QPID_LOG(debug, "CyrusSasl::start(" << mechanisms << ")");
     int result = sasl_client_new(settings.service.c_str(),
@@ -189,7 +189,18 @@
     if (result != SASL_OK) throw InternalErrorException(QPID_MSG("Sasl error: " << sasl_errdetail(conn)));
 
     sasl_security_properties_t secprops;
-    
+
+    if (ssf) {
+        sasl_ssf_t external_ssf = (sasl_ssf_t) ssf;
+        if (external_ssf) {
+            int result = sasl_setprop(conn, SASL_SSF_EXTERNAL, &external_ssf);
+            if (result != SASL_OK) {
+                throw framing::InternalErrorException(QPID_MSG("SASL error: unable to set external SSF: " << result));
+            }
+            QPID_LOG(debug, "external SSF detected and set to " << ssf);
+        }
+    }
+
     secprops.min_ssf = settings.minSsf;
     secprops.max_ssf = settings.maxSsf;
     secprops.maxbufsize = 65535;

Modified: qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp Mon Nov  9 15:30:18 2009
@@ -135,6 +135,7 @@
     SslConnector(framing::ProtocolVersion pVersion,
               const ConnectionSettings&, 
               ConnectionImpl*);
+    unsigned int getSSF() { return socket.getKeyLen(); }
 };
 
 // Static constructor which registers connector here

Modified: qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp Mon Nov  9 15:30:18 2009
@@ -153,7 +153,7 @@
  * Currently use SVN revision to avoid clashes with versions from
  * different branches.
  */
-const uint32_t Cluster::CLUSTER_VERSION = 820783;
+const uint32_t Cluster::CLUSTER_VERSION = 834052;
 
 struct ClusterDispatcher : public framing::AMQP_AllOperations::ClusterHandler {
     qpid::cluster::Cluster& cluster;
@@ -250,7 +250,9 @@
     assert(c->getId().getMember() == self);
     // Announce the connection to the cluster.
     if (c->isLocalClient())
-        mcast.mcastControl((ClusterConnectionAnnounceBody()), c->getId());
+        mcast.mcastControl(ClusterConnectionAnnounceBody(ProtocolVersion(),
+                                                         c->getBrokerConnection().getSSF() ),
+                           c->getId());
 }
 
 // Called in connection thread to insert an updated shadow connection.
@@ -344,7 +346,13 @@
              body->getMethod()->isA<ClusterUpdateOfferBody>()) ?
         static_cast<const ClusterUpdateOfferBody*>(body) : 0;
 }
- 
+
+const ClusterConnectionAnnounceBody* castAnnounce( const framing::AMQBody *body) {
+    return  (body && body->getMethod() &&
+             body->getMethod()->isA<ClusterConnectionAnnounceBody>()) ?
+      static_cast<const ClusterConnectionAnnounceBody*>(body) : 0;
+}
+
 // Handler for deliverEventQueue.
 // This thread decodes frames from events.
 void Cluster::deliveredEvent(const Event& e) {
@@ -452,8 +460,13 @@
         }
         else {              // New remote connection, create a shadow.
             std::ostringstream mgmtId;
+            unsigned int ssf;
+            const ClusterConnectionAnnounceBody *announce = castAnnounce(e.frame.getBody());
+
             mgmtId << id;
-            cp = new Connection(*this, shadowOut, mgmtId.str(), id);
+            ssf = (announce && announce->hasSsf()) ? announce->getSsf() : 0;
+            QPID_LOG(debug, *this << "new connection's ssf =" << ssf );
+            cp = new Connection(*this, shadowOut, mgmtId.str(), id, ssf );
         }
         connections.insert(ConnectionMap::value_type(id, cp));
     }

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=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp Mon Nov  9 15:30:18 2009
@@ -72,9 +72,10 @@
 
 
 // Shadow connection
-Connection::Connection(Cluster& c, sys::ConnectionOutputHandler& out, const std::string& logId, const ConnectionId& id)
+    Connection::Connection(Cluster& c, sys::ConnectionOutputHandler& out, const std::string& logId,
+                           const ConnectionId& id, unsigned int ssf)
     : cluster(c), self(id), catchUp(false), output(*this, out),
-      connection(&output, cluster.getBroker(), shadowPrefix+logId), expectProtocolHeader(false),
+      connection(&output, cluster.getBroker(), shadowPrefix+logId, ssf), expectProtocolHeader(false),
       mcastFrameHandler(cluster.getMulticast(), self),
       consumerNumbering(c.getUpdateReceiver().consumerNumbering)
 { init(); }
@@ -82,10 +83,11 @@
 // Local connection
 Connection::Connection(Cluster& c, sys::ConnectionOutputHandler& out,
                        const std::string& logId, MemberId member,
-                       bool isCatchUp, bool isLink
+                       bool isCatchUp, bool isLink, unsigned int ssf
 ) : cluster(c), self(member, ++idCounter), catchUp(isCatchUp), output(*this, out),
     connection(&output, cluster.getBroker(),
                isCatchUp ? shadowPrefix+logId : logId,
+               ssf,
                isLink,
                isCatchUp ? ++catchUpId : 0),
     expectProtocolHeader(isLink), mcastFrameHandler(cluster.getMulticast(), self),

Modified: qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h Mon Nov  9 15:30:18 2009
@@ -63,10 +63,13 @@
         
 {
   public:
+    
     /** Local connection. */
-    Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& logId, MemberId, bool catchUp, bool isLink);
+    Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& logId, MemberId, bool catchUp, bool isLink,
+               unsigned int ssf);
     /** Shadow connection. */
-    Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& logId, const ConnectionId& id);
+    Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& logId, const ConnectionId& id,
+               unsigned int ssf);
     ~Connection();
     
     ConnectionId getId() const { return self; }
@@ -155,7 +158,7 @@
     void exchange(const std::string& encoded);
 
     void giveReadCredit(int credit);
-    void announce() {}          // handled by Cluster.
+    void announce(uint32_t) {}  // handled by Cluster.
     void abort();
     void deliverClose();
 

Modified: qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp Mon Nov  9 15:30:18 2009
@@ -36,25 +36,27 @@
 using namespace framing;
 
 sys::ConnectionCodec*
-ConnectionCodec::Factory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id) {
+ConnectionCodec::Factory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
+                                 unsigned int ssf) {
     if (v == ProtocolVersion(0, 10))
-        return new ConnectionCodec(v, out, id, cluster, false, false);
+        return new ConnectionCodec(v, out, id, cluster, false, false, ssf);
     else if (v == ProtocolVersion(0x80 + 0, 0x80 + 10)) // Catch-up connection
-        return new ConnectionCodec(v, out, id, cluster, true, false); 
+        return new ConnectionCodec(v, out, id, cluster, true, false, ssf); 
     return 0;
 }
 
 // Used for outgoing Link connections
 sys::ConnectionCodec*
-ConnectionCodec::Factory::create(sys::OutputControl& out, const std::string& logId) {
-    return new ConnectionCodec(ProtocolVersion(0,10), out, logId, cluster, false, true);
+ConnectionCodec::Factory::create(sys::OutputControl& out, const std::string& logId,
+                                 unsigned int ssf) {
+    return new ConnectionCodec(ProtocolVersion(0,10), out, logId, cluster, false, true, ssf);
 }
 
 ConnectionCodec::ConnectionCodec(
     const ProtocolVersion& v, sys::OutputControl& out,
-    const std::string& logId, Cluster& cluster, bool catchUp, bool isLink
+    const std::string& logId, Cluster& cluster, bool catchUp, bool isLink, unsigned int ssf
 ) : codec(out, logId, isLink),
-    interceptor(new Connection(cluster, codec, logId, cluster.getId(), catchUp, isLink))
+    interceptor(new Connection(cluster, codec, logId, cluster.getId(), catchUp, isLink, ssf))
 {
     std::auto_ptr<sys::ConnectionInputHandler> ih(new ProxyInputHandler(interceptor));
     codec.setInputHandler(ih);

Modified: qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h Mon Nov  9 15:30:18 2009
@@ -52,12 +52,15 @@
         Cluster& cluster;
         Factory(boost::shared_ptr<sys::ConnectionCodec::Factory> f, Cluster& c)
             : next(f), cluster(c) {}
-        sys::ConnectionCodec* create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id);
-        sys::ConnectionCodec* create(sys::OutputControl&, const std::string& id);
+        sys::ConnectionCodec* create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id,
+                                     unsigned int conn_ssf);
+        sys::ConnectionCodec* create(sys::OutputControl&, const std::string& id,
+                                     unsigned int conn_ssf);
     };
 
     ConnectionCodec(const framing::ProtocolVersion&, sys::OutputControl& out,
-                    const std::string& logId, Cluster& c, bool catchUp, bool isLink);
+                    const std::string& logId, Cluster& c, bool catchUp, bool isLink,
+                    unsigned int ssf);
     ~ConnectionCodec();
 
     // ConnectionCodec functions.

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp Mon Nov  9 15:30:18 2009
@@ -144,7 +144,7 @@
             decoded = in.getPosition();
             QPID_LOG(debug, "RECV [" << identifier << "] INIT(" << protocolInit << ")");
             try {
-                codec = factory->create(protocolInit.getVersion(), *this, identifier);
+                codec = factory->create(protocolInit.getVersion(), *this, identifier, 0);
                 if (!codec) {
                     //TODO: may still want to revise this...
                     //send valid version header & close connection.
@@ -200,7 +200,7 @@
 
 void AsynchIOHandler::idle(AsynchIO&){
     if (isClient && codec == 0) {
-        codec = factory->create(*this, identifier);
+        codec = factory->create(*this, identifier, 0);
         write(framing::ProtocolInitiation(codec->getVersion()));
         return;
     }

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h Mon Nov  9 15:30:18 2009
@@ -45,18 +45,31 @@
     virtual bool isClosed() const = 0;
 
     virtual framing::ProtocolVersion getVersion() const = 0;
-    
+
     struct Factory {
         virtual ~Factory() {}
 
+        /** Security Strength Factor - indicates the level of security provided
+         * by the underlying transport.  If zero, the transport provides no
+         * security (e.g. TCP). If non-zero, the transport provides some level
+         * of security (e.g. SSL).  The values for SSF can be interpreted as:
+         *
+         * 0 = No protection.
+         * 1 = Integrity checking only.
+         * >1 = Supports authentication, integrity and confidentiality.
+         *      The number represents the encryption key length.
+         */
+
         /** Return 0 if version unknown */
         virtual ConnectionCodec* create(
-            framing::ProtocolVersion, OutputControl&, const std::string& id
+            framing::ProtocolVersion, OutputControl&, const std::string& id,
+            unsigned int conn_ssf
         ) = 0;
 
         /** Return "preferred" codec for outbound connections. */
         virtual ConnectionCodec* create(
-            OutputControl&, const std::string& id
+            OutputControl&, const std::string& id,
+            unsigned int conn_ssf
         ) = 0;
     };
 };

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp Mon Nov  9 15:30:18 2009
@@ -139,7 +139,7 @@
     // but we must be able to send
     assert( codec == 0 );
     assert( aio->writable() && aio->bufferAvailable() );
-    codec = factory->create(*this, identifier);
+    codec = factory->create(*this, identifier, 0);
     write(framing::ProtocolInitiation(codec->getVersion()));
 }
 
@@ -186,7 +186,7 @@
         decoded = in.getPosition();
         QPID_LOG(debug, "Rdma: RECV [" << identifier << "] INIT(" << protocolInit << ")");
 
-        codec = factory->create(protocolInit.getVersion(), *this, identifier);
+        codec = factory->create(protocolInit.getVersion(), *this, identifier, 0);
 
         // If we failed to create the codec then we don't understand the offered protocol version
         if (!codec) {

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp Mon Nov  9 15:30:18 2009
@@ -111,7 +111,7 @@
             decoded = in.getPosition();
             QPID_LOG(debug, "RECV [" << identifier << "] INIT(" << protocolInit << ")");
             try {
-                codec = factory->create(protocolInit.getVersion(), *this, identifier);
+                codec = factory->create(protocolInit.getVersion(), *this, identifier, aio->getKeyLen());
                 if (!codec) {
                     //TODO: may still want to revise this...
                     //send valid version header & close connection.
@@ -166,7 +166,7 @@
 
 void SslHandler::idle(SslIO&){
     if (isClient && codec == 0) {
-        codec = factory->create(*this, identifier);
+        codec = factory->create(*this, identifier, aio->getKeyLen());
         write(framing::ProtocolInitiation(codec->getVersion()));
         return;
     }

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp Mon Nov  9 15:30:18 2009
@@ -436,3 +436,4 @@
     }
 }
 
+int SslIO::getKeyLen() {return socket.getKeyLen();}

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h Mon Nov  9 15:30:18 2009
@@ -121,6 +121,7 @@
     typedef boost::function1<void, SslIO&> BuffersEmptyCallback;
     typedef boost::function1<void, SslIO&> IdleCallback;
 
+
 private:
     ReadCallback readCallback;
     EofCallback eofCallback;
@@ -155,6 +156,8 @@
     bool writeQueueEmpty() { return writeQueue.empty(); }
     BufferBase* getQueuedBuffer();
 
+    int getKeyLen();
+
 private:
     ~SslIO();
     void readable(qpid::sys::DispatchHandle& handle);

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp Mon Nov  9 15:30:18 2009
@@ -274,4 +274,24 @@
     }
 }
 
+
+/** get the bit length of the current cipher's key */
+int SslSocket::getKeyLen() const
+{
+    int enabled = 0;
+    int keySize = 0;
+    SECStatus   rc;
+
+    rc = SSL_SecurityStatus( socket,
+                             &enabled,
+                             NULL,
+                             NULL,
+                             &keySize,
+                             NULL, NULL );
+    if (rc == SECSuccess && enabled) {
+        return keySize;
+    }
+    return 0;
+}
+
 }}} // namespace qpid::sys::ssl

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h Mon Nov  9 15:30:18 2009
@@ -100,6 +100,8 @@
      */
     int getError() const;
 
+    int getKeyLen() const;
+
 private:
     mutable std::string connectname;
     mutable PRFileDesc* socket;

Modified: qpid/trunk/qpid/cpp/xml/cluster.xml
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/xml/cluster.xml?rev=834108&r1=834107&r2=834108&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/xml/cluster.xml (original)
+++ qpid/trunk/qpid/cpp/xml/cluster.xml Mon Nov  9 15:30:18 2009
@@ -81,7 +81,12 @@
   <class name="cluster-connection" code="0x81" label="Qpid clustering extensions.">
 
     <!-- Announce a new connection -->
-    <control name="announce" code="0x1"/>
+    <control name="announce" code="0x1">
+      <!-- Security Strength Factor (ssf): if the transport provides
+      encryption (e.g. ssl), ssf is the bit length of the key.  Zero if no
+      encryption provided. -->
+      <field name="ssf" type="uint32"/>
+    </control>
 
     <!-- Marks the cluster-wide point when a connection is considered closed. -->
     <control name="deliver-close" code="0x2"/>



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