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 2010/03/05 17:51:23 UTC

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

Author: gsim
Date: Fri Mar  5 16:51:22 2010
New Revision: 919487

URL: http://svn.apache.org/viewvc?rev=919487&view=rev
Log:
QPID-2412: Support for EXTERNAL mechanism on client-authenticated SSL connections.

On SSL connection where the clients certificate is authenticated (requires the --ssl-require-client-authentication option at present), the clients identity will be taken from that certificate (it will be the CN with any DCs present appended as the domain, e.g. CN=bob,DC=acme,DC=com would result in an identity of bob@acme.com). This will enable the EXTERNAL mechanism when cyrus sasl is in use.

The client can still negotiate their desired mechanism. There is a new option on the ssl module (--ssl-sasl-no-dict) that allows the options on ssl connections to be restricted to those that are not vulnerable to dictionary attacks (EXTERNAL being the primary example).


Added:
    qpid/trunk/qpid/cpp/src/qpid/sys/SecuritySettings.h
Modified:
    qpid/trunk/qpid/cpp/src/Makefile.am
    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.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/client/TCPConnector.h
    qpid/trunk/qpid/cpp/src/qpid/client/windows/SaslFactory.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/SslPlugin.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp
    qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.h
    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/Makefile.am
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/Makefile.am?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ qpid/trunk/qpid/cpp/src/Makefile.am Fri Mar  5 16:51:22 2010
@@ -463,6 +463,7 @@
   qpid/sys/Runnable.cpp				\
   qpid/sys/ScopedIncrement.h			\
   qpid/sys/SecurityLayer.h			\
+  qpid/sys/SecuritySettings.h			\
   qpid/sys/Semaphore.h				\
   qpid/sys/Shlib.cpp				\
   qpid/sys/Shlib.h				\

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/Connection.cpp Fri Mar  5 16:51:22 2010
@@ -23,6 +23,7 @@
 #include "qpid/broker/SessionState.h"
 #include "qpid/broker/Bridge.h"
 #include "qpid/broker/Broker.h"
+#include "qpid/sys/SecuritySettings.h"
 
 #include "qpid/log/Statement.h"
 #include "qpid/ptr_map.h"
@@ -72,9 +73,10 @@
     }
 };
 
-Connection::Connection(ConnectionOutputHandler* out_, Broker& broker_, const std::string& mgmtId_, unsigned int ssf, bool isLink_, uint64_t objectId, bool shadow_) :
+Connection::Connection(ConnectionOutputHandler* out_, Broker& broker_, const std::string& mgmtId_,
+                       const qpid::sys::SecuritySettings& external, bool isLink_, uint64_t objectId, bool shadow_) :
     ConnectionState(out_, broker_),
-    ssf(ssf),
+    securitySettings(external),
     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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/Connection.h Fri Mar  5 16:51:22 2010
@@ -45,6 +45,7 @@
 #include "qpid/sys/AggregateOutput.h"
 #include "qpid/sys/ConnectionInputHandler.h"
 #include "qpid/sys/ConnectionOutputHandler.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/sys/Socket.h"
 #include "qpid/sys/TimeoutHandler.h"
 #include "qpid/sys/Mutex.h"
@@ -78,7 +79,8 @@
         virtual void connectionError(const std::string&) = 0;
     };
 
-    Connection(sys::ConnectionOutputHandler* out, Broker& broker, const std::string& mgmtId, unsigned int ssf,
+    Connection(sys::ConnectionOutputHandler* out, Broker& broker, const std::string& mgmtId,
+               const qpid::sys::SecuritySettings&,
                bool isLink = false, uint64_t objectId = 0, bool shadow=false);
     ~Connection ();
 
@@ -136,14 +138,17 @@
     // Used by cluster to update connection status
     sys::AggregateOutput& getOutputTasks() { return outputTasks; }
 
-    unsigned int getSSF() { return ssf; }
+    const qpid::sys::SecuritySettings& getExternalSecuritySettings() const
+    { 
+        return securitySettings;
+    }
 
   private:
     typedef boost::ptr_map<framing::ChannelId, SessionHandler> ChannelMap;
     typedef std::vector<Queue::shared_ptr>::iterator queue_iterator;
 
     ChannelMap channels;
-    unsigned int ssf;
+    qpid::sys::SecuritySettings securitySettings;
     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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.cpp Fri Mar  5 16:51:22 2010
@@ -22,12 +22,14 @@
 #include "qpid/framing/ProtocolVersion.h"
 #include "qpid/amqp_0_10/Connection.h"
 #include "qpid/broker/Connection.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/log/Statement.h"
 
 namespace qpid {
 namespace broker {
 
 using framing::ProtocolVersion;
+using qpid::sys::SecuritySettings;
 typedef std::auto_ptr<amqp_0_10::Connection> ConnectionPtr;
 typedef std::auto_ptr<sys::ConnectionInputHandler> InputPtr;
 
@@ -37,7 +39,7 @@
 
 sys::ConnectionCodec*
 ConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
-                          unsigned int ) {
+                          const SecuritySettings& external) {
     if (broker.getConnectionCounter().allowConnection())
     {
         QPID_LOG(error, "Client max connection count limit exceeded: " << broker.getOptions().maxConnections << " connection refused");
@@ -45,7 +47,7 @@
     }
     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)));
+        c->setInputHandler(InputPtr(new broker::Connection(c.get(), broker, id, external, false)));
         return c.release();
     }
     return 0;
@@ -53,10 +55,10 @@
 
 sys::ConnectionCodec*
 ConnectionFactory::create(sys::OutputControl& out, const std::string& id,
-                          unsigned int) {
+                          const SecuritySettings& external) {
     // 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)));
+    c->setInputHandler(InputPtr(new broker::Connection(c.get(), broker, id, external, true)));
     return c.release();
 }
 

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/ConnectionFactory.h Fri Mar  5 16:51:22 2010
@@ -36,11 +36,10 @@
 
     sys::ConnectionCodec*
     create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id,
-           unsigned int conn_ssf);
+           const qpid::sys::SecuritySettings&);
 
     sys::ConnectionCodec*
-    create(sys::OutputControl&, const std::string& id,
-           unsigned int conn_ssf);
+    create(sys::OutputControl&, const std::string& id, const qpid::sys::SecuritySettings&);
 
   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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SaslAuthenticator.cpp Fri Mar  5 16:51:22 2010
@@ -26,6 +26,7 @@
 #include "qpid/broker/Connection.h"
 #include "qpid/log/Statement.h"
 #include "qpid/framing/reply_exceptions.h"
+#include "qpid/sys/SecuritySettings.h"
 #include <boost/format.hpp>
 
 #if HAVE_SASL
@@ -36,6 +37,7 @@
 
 using namespace qpid::framing;
 using qpid::sys::SecurityLayer;
+using qpid::sys::SecuritySettings;
 using boost::format;
 using boost::str;
 
@@ -152,7 +154,8 @@
 #if HAVE_SASL
         // encryption required - check to see if we are running over an
         // encrypted SSL connection.
-        sasl_ssf_t external_ssf = (sasl_ssf_t) connection.getSSF();
+        SecuritySettings external = connection.getExternalSecuritySettings();
+        sasl_ssf_t external_ssf = (sasl_ssf_t) external.ssf;
         if (external_ssf < 1)    // < 1 == unencrypted
 #endif
         {
@@ -244,7 +247,9 @@
 
     // 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();
+    SecuritySettings external = connection.getExternalSecuritySettings();
+    QPID_LOG(debug, "External ssf=" << external.ssf << " and auth=" << external.authid);
+    sasl_ssf_t external_ssf = (sasl_ssf_t) external.ssf;
     if (external_ssf) {
         int result = sasl_setprop(sasl_conn, SASL_SSF_EXTERNAL, &external_ssf);
         if (result != SASL_OK) {
@@ -258,16 +263,25 @@
              ", max_ssf: " << secprops.max_ssf <<
              ", external_ssf: " << external_ssf );
 
+    if (!external.authid.empty()) {
+        const char* external_authid = external.authid.c_str();
+        int result = sasl_setprop(sasl_conn, SASL_AUTH_EXTERNAL, external_authid);
+        if (result != SASL_OK) {
+            throw framing::InternalErrorException(QPID_MSG("SASL error: unable to set external auth: " << result));
+        }
+
+        QPID_LOG(debug, "external auth detected and set to " << external_authid);
+    }
+
     secprops.maxbufsize = 65535;
     secprops.property_names = 0;
     secprops.property_values = 0;
     secprops.security_flags = 0; /* or SASL_SEC_NOANONYMOUS etc as appropriate */
-    
+    if (external.nodict) secprops.security_flags |= SASL_SEC_NODICTIONARY;    
     int result = sasl_setprop(sasl_conn, SASL_SEC_PROPS, &secprops);
     if (result != SASL_OK) {
         throw framing::InternalErrorException(QPID_MSG("SASL error: " << result));
     }
-
 }
 
 CyrusAuthenticator::~CyrusAuthenticator()

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.cpp Fri Mar  5 16:51:22 2010
@@ -23,12 +23,14 @@
 #include "qpid/amqp_0_10/Connection.h"
 #include "qpid/broker/Connection.h"
 #include "qpid/broker/SecureConnection.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/log/Statement.h"
 
 namespace qpid {
 namespace broker {
 
 using framing::ProtocolVersion;
+using qpid::sys::SecuritySettings;
 typedef std::auto_ptr<amqp_0_10::Connection> CodecPtr;
 typedef std::auto_ptr<SecureConnection> SecureConnectionPtr;
 typedef std::auto_ptr<Connection> ConnectionPtr;
@@ -38,7 +40,7 @@
 
 sys::ConnectionCodec*
 SecureConnectionFactory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
-                                unsigned int conn_ssf ) {
+                                const SecuritySettings& external) {
     if (broker.getConnectionCounter().allowConnection())
     {
         QPID_LOG(error, "Client max connection count limit exceeded: " << broker.getOptions().maxConnections << " connection refused");
@@ -47,7 +49,7 @@
     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, conn_ssf, false));
+        ConnectionPtr i(new broker::Connection(c.get(), broker, id, external, false));
         i->setSecureConnection(sc.get());
         c->setInputHandler(InputPtr(i.release()));
         sc->setCodec(std::auto_ptr<sys::ConnectionCodec>(c));
@@ -58,11 +60,11 @@
 
 sys::ConnectionCodec*
 SecureConnectionFactory::create(sys::OutputControl& out, const std::string& id,
-                                unsigned int conn_ssf) {
+                                const SecuritySettings& external) {
     // 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, conn_ssf, true ));
+    ConnectionPtr i(new broker::Connection(c.get(), broker, id, external, 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/broker/SecureConnectionFactory.h Fri Mar  5 16:51:22 2010
@@ -34,11 +34,10 @@
 
     sys::ConnectionCodec*
     create(framing::ProtocolVersion, sys::OutputControl&, const std::string& id,
-           unsigned int conn_ssf);
+           const qpid::sys::SecuritySettings&);
 
     sys::ConnectionCodec*
-    create(sys::OutputControl&, const std::string& id,
-           unsigned int conn_ssf);
+    create(sys::OutputControl&, const std::string& id, const qpid::sys::SecuritySettings&);
 
   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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.cpp Fri Mar  5 16:51:22 2010
@@ -213,7 +213,7 @@
 
     if (sasl.get()) {
         string response = sasl->start(mechanism.empty() ? mechlist : mechanism,
-                                      getSSF ? getSSF() : 0);
+                                      getSecuritySettings ? getSecuritySettings() : 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionHandler.h Fri Mar  5 16:51:22 2010
@@ -40,6 +40,11 @@
 #include <memory>
 
 namespace qpid {
+
+namespace sys {
+struct SecuritySettings;
+}
+
 namespace client {
 
 class ConnectionHandler : private StateManager,
@@ -95,7 +100,7 @@
     using InputHandler::handle;
     typedef boost::function<void()> CloseListener;    
     typedef boost::function<void(uint16_t, const std::string&)> ErrorListener;
-    typedef boost::function<unsigned int()> GetConnSSF;
+    typedef boost::function<const qpid::sys::SecuritySettings*()> GetSecuritySettings;
 
     ConnectionHandler(const ConnectionSettings&, framing::ProtocolVersion&);
 
@@ -123,7 +128,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 */
+    GetSecuritySettings  getSecuritySettings;     /** query the transport for its security details */
 };
 
 }}

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/ConnectionImpl.cpp Fri Mar  5 16:51:22 2010
@@ -165,7 +165,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));
+    handler.getSecuritySettings = boost::bind(&Connector::getSecuritySettings, boost::ref(connector));
 }
 
 const uint16_t ConnectionImpl::NEXT_CHANNEL = std::numeric_limits<uint16_t>::max();

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/Connector.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/Connector.h Fri Mar  5 16:51:22 2010
@@ -35,6 +35,7 @@
 class ShutdownHandler;
 class SecurityLayer;
 class Poller;
+struct SecuritySettings;
 }
 
 namespace framing {
@@ -74,7 +75,7 @@
 
     virtual void activateSecurityLayer(std::auto_ptr<qpid::sys::SecurityLayer>);
 
-    virtual unsigned int getSSF() = 0;
+    virtual const qpid::sys::SecuritySettings* getSecuritySettings() = 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/RdmaConnector.cpp Fri Mar  5 16:51:22 2010
@@ -109,7 +109,7 @@
     framing::OutputHandler* getOutputHandler();
     const std::string& getIdentifier() const;
     void activateSecurityLayer(std::auto_ptr<qpid::sys::SecurityLayer>);
-    unsigned int getSSF() { return 0; }
+    const qpid::sys::SecuritySettings* getSecuritySettings() { return 0; }
 
     size_t decode(const char* buffer, size_t size);
     size_t encode(const char* buffer, size_t size);

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/Sasl.h Fri Mar  5 16:51:22 2010
@@ -30,6 +30,7 @@
 
 namespace sys {
 class SecurityLayer;
+struct SecuritySettings;
 }
 
 namespace client {
@@ -48,17 +49,10 @@
      *
      * @param mechanisms Comma-separated list of the SASL mechanism the
      *             client supports.
-     * @param ssf  Security Strength Factor (SSF). SSF is used to negotiate
-     *             a SASL security layer on top of the connection should both
-     *             parties require and support it. The value indicates the
-     *             required level of security for communication. Possible
-     *             values are:
-     *             @li 0  No security
-     *             @li 1  Integrity checking only
-     *             @li >1 Integrity and confidentiality with the number
-     *                    giving the encryption key length.
+     * @param externalSecuritySettings security related details from the underlying transport
      */
-    virtual std::string start(const std::string& mechanisms, unsigned int ssf) = 0;
+    virtual std::string start(const std::string& mechanisms,
+                              const qpid::sys::SecuritySettings* externalSecuritySettings = 0) = 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/SaslFactory.cpp Fri Mar  5 16:51:22 2010
@@ -61,6 +61,7 @@
 #include "qpid/Exception.h"
 #include "qpid/framing/reply_exceptions.h"
 #include "qpid/sys/SecurityLayer.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/sys/cyrus/CyrusSecurityLayer.h"
 #include "qpid/log/Statement.h"
 #include <sasl/sasl.h>
@@ -70,6 +71,7 @@
 namespace client {
 
 using qpid::sys::SecurityLayer;
+using qpid::sys::SecuritySettings;
 using qpid::sys::cyrus::CyrusSecurityLayer;
 using qpid::framing::InternalErrorException;
 
@@ -80,7 +82,7 @@
   public:
     CyrusSasl(const ConnectionSettings&);
     ~CyrusSasl();
-    std::string start(const std::string& mechanisms, unsigned int ssf);
+    std::string start(const std::string& mechanisms, const SecuritySettings* externalSettings);
     std::string step(const std::string& challenge);
     std::string getMechanism();
     std::string getUserId();
@@ -176,7 +178,7 @@
     const std::string SSL("ssl");
 }
 
-std::string CyrusSasl::start(const std::string& mechanisms, unsigned int ssf)
+std::string CyrusSasl::start(const std::string& mechanisms, const SecuritySettings* externalSettings)
 {
     QPID_LOG(debug, "CyrusSasl::start(" << mechanisms << ")");
     int result = sasl_client_new(settings.service.c_str(),
@@ -190,14 +192,22 @@
 
     sasl_security_properties_t secprops;
 
-    if (ssf) {
-        sasl_ssf_t external_ssf = (sasl_ssf_t) ssf;
+    if (externalSettings) {
+        sasl_ssf_t external_ssf = (sasl_ssf_t) externalSettings->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);
+            QPID_LOG(debug, "external SSF detected and set to " << external_ssf);
+        }
+        if (externalSettings->authid.size()) {
+            const char* external_authid = externalSettings->authid.c_str();
+            result = sasl_setprop(conn, SASL_AUTH_EXTERNAL, external_authid);
+            if (result != SASL_OK) {
+                throw framing::InternalErrorException(QPID_MSG("SASL error: unable to set external auth: " << result));
+            }
+            QPID_LOG(debug, "external auth detected and set to " << external_authid);
         }
     }
 
@@ -216,7 +226,6 @@
         throw framing::InternalErrorException(QPID_MSG("SASL error: " << sasl_errdetail(conn)));
     }
 
-
     sasl_interact_t* client_interact = 0;
     const char *out = 0;
     unsigned outlen = 0;

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/SslConnector.cpp Fri Mar  5 16:51:22 2010
@@ -34,6 +34,7 @@
 #include "qpid/sys/ssl/SslSocket.h"
 #include "qpid/sys/Dispatcher.h"
 #include "qpid/sys/Poller.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/Msg.h"
 
 #include <iostream>
@@ -86,6 +87,7 @@
     const uint16_t maxFrameSize;
     framing::ProtocolVersion version;
     bool initiated;
+    SecuritySettings securitySettings;
 
     sys::Mutex closedLock;
     bool closed;
@@ -125,7 +127,7 @@
     sys::ShutdownHandler* getShutdownHandler() const;
     framing::OutputHandler* getOutputHandler();
     const std::string& getIdentifier() const;
-    unsigned int getSSF() { return socket.getKeyLen(); }
+    const SecuritySettings* getSecuritySettings();
 
 public:
     SslConnector(Poller::shared_ptr p, framing::ProtocolVersion pVersion,
@@ -366,4 +368,11 @@
     handleClosed();
 }
 
+const SecuritySettings* SslConnector::getSecuritySettings()
+{
+    securitySettings.ssf = socket.getKeyLen();
+    securitySettings.authid = "dummy";//set to non-empty string to enable external authentication
+    return &securitySettings; 
+}
+
 }} // namespace qpid::client

Modified: qpid/trunk/qpid/cpp/src/qpid/client/TCPConnector.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/TCPConnector.h?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/TCPConnector.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/TCPConnector.h Fri Mar  5 16:51:22 2010
@@ -92,7 +92,7 @@
     framing::OutputHandler* getOutputHandler();
     const std::string& getIdentifier() const;
     void activateSecurityLayer(std::auto_ptr<qpid::sys::SecurityLayer>);
-    unsigned int getSSF() { return 0; }
+    const qpid::sys::SecuritySettings* getSecuritySettings() { return 0; }
 
     size_t decode(const char* buffer, size_t size);
     size_t encode(const char* buffer, size_t size);

Modified: qpid/trunk/qpid/cpp/src/qpid/client/windows/SaslFactory.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/client/windows/SaslFactory.cpp?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/client/windows/SaslFactory.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/client/windows/SaslFactory.cpp Fri Mar  5 16:51:22 2010
@@ -25,6 +25,7 @@
 #include "qpid/Exception.h"
 #include "qpid/framing/reply_exceptions.h"
 #include "qpid/sys/SecurityLayer.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/log/Statement.h"
 
 #include "boost/tokenizer.hpp"
@@ -33,6 +34,7 @@
 namespace client {
 
 using qpid::sys::SecurityLayer;
+using qpid::sys::SecuritySettings;
 using qpid::framing::InternalErrorException;
 
 class WindowsSasl : public Sasl
@@ -40,7 +42,7 @@
   public:
     WindowsSasl(const ConnectionSettings&);
     ~WindowsSasl();
-    std::string start(const std::string& mechanisms, unsigned int ssf);
+    std::string start(const std::string& mechanisms, const SecuritySettings* externalSettings);
     std::string step(const std::string& challenge);
     std::string getMechanism();
     std::string getUserId();
@@ -91,7 +93,7 @@
 }
 
 std::string WindowsSasl::start(const std::string& mechanisms,
-                               unsigned int /*ssf*/)
+                               const SecuritySettings* /*externalSettings*/)
 {
     QPID_LOG(debug, "WindowsSasl::start(" << mechanisms << ")");
 

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp Fri Mar  5 16:51:22 2010
@@ -510,8 +510,13 @@
             assert(cp);
         }
         else {              // New remote connection, create a shadow.
-            unsigned int ssf = (announce && announce->hasSsf()) ? announce->getSsf() : 0;
-            cp = new Connection(*this, shadowOut, announce->getManagementId(), id, ssf);
+            qpid::sys::SecuritySettings secSettings;
+            if (announce) {
+                secSettings.ssf = announce->getSsf();
+                secSettings.authid = announce->getAuthid();
+                secSettings.nodict = announce->getNodict();
+            }
+            cp = new Connection(*this, shadowOut, announce->getManagementId(), id, secSettings);
         }
         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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.cpp Fri Mar  5 16:51:22 2010
@@ -77,9 +77,9 @@
 // Shadow connection
 Connection::Connection(Cluster& c, sys::ConnectionOutputHandler& out,
                        const std::string& mgmtId,
-                       const ConnectionId& id, unsigned int ssf)
+                       const ConnectionId& id, const qpid::sys::SecuritySettings& external)
     : cluster(c), self(id), catchUp(false), output(*this, out),
-      connectionCtor(&output, cluster.getBroker(), mgmtId, ssf, false, 0, true),
+      connectionCtor(&output, cluster.getBroker(), mgmtId, external, false, 0, true),
       expectProtocolHeader(false),
       mcastFrameHandler(cluster.getMulticast(), self),
       updateIn(c.getUpdateReceiver())
@@ -88,11 +88,11 @@
 // Local connection
 Connection::Connection(Cluster& c, sys::ConnectionOutputHandler& out,
                        const std::string& mgmtId, MemberId member,
-                       bool isCatchUp, bool isLink, unsigned int ssf
+                       bool isCatchUp, bool isLink, const qpid::sys::SecuritySettings& external
 ) : cluster(c), self(member, ++idCounter), catchUp(isCatchUp), output(*this, out),
     connectionCtor(&output, cluster.getBroker(),
                    mgmtId,
-                   ssf,
+                   external,
                    isLink,
                    isCatchUp ? ++catchUpId : 0,
                    isCatchUp),  // isCatchUp => shadow
@@ -107,7 +107,10 @@
         QPID_LOG(info, "new client connection " << *this);
         giveReadCredit(cluster.getSettings().readMax);
         cluster.getMulticast().mcastControl(
-            ClusterConnectionAnnounceBody(ProtocolVersion(), mgmtId, getSsf()), getId());
+            ClusterConnectionAnnounceBody(ProtocolVersion(), mgmtId,
+                                          connectionCtor.external.ssf,
+                                          connectionCtor.external.authid,
+                                          connectionCtor.external.nodict), getId());
     }
     else {
         // Catch-up shadow connections initialized using nextShadow id.
@@ -122,7 +125,7 @@
 void Connection::init() {
     connection = connectionCtor.construct();
     QPID_LOG(debug, cluster << " initialized connection: " << *this
-             << " ssf=" << connection->getSSF());
+             << " ssf=" << connection->getExternalSecuritySettings().ssf);
     if (isLocalClient()) {  
         // Actively send cluster-order frames from local node
         connection->setClusterOrderOutput(mcastFrameHandler);
@@ -142,9 +145,11 @@
         output.giveReadCredit(credit);
 }
 
-void Connection::announce(const std::string& mgmtId, uint32_t ssf) {
+void Connection::announce(const std::string& mgmtId, uint32_t ssf, const std::string& authid, bool nodict) {
     QPID_ASSERT(mgmtId == connectionCtor.mgmtId);
-    QPID_ASSERT(ssf == connectionCtor.ssf);
+    QPID_ASSERT(ssf == connectionCtor.external.ssf);
+    QPID_ASSERT(authid == connectionCtor.external.authid);
+    QPID_ASSERT(nodict == connectionCtor.external.nodict);
     init();
 }
 

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/Connection.h Fri Mar  5 16:51:22 2010
@@ -34,6 +34,7 @@
 #include "qpid/sys/AtomicValue.h"
 #include "qpid/sys/ConnectionInputHandler.h"
 #include "qpid/sys/ConnectionOutputHandler.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/framing/SequenceNumber.h"
 #include "qpid/framing/FrameDecoder.h"
 
@@ -66,10 +67,10 @@
     
     /** Local connection. */
     Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& mgmtId, MemberId, bool catchUp, bool isLink,
-               unsigned int ssf);
+               const qpid::sys::SecuritySettings& external);
     /** Shadow connection. */
     Connection(Cluster&, sys::ConnectionOutputHandler& out, const std::string& mgmtId, const ConnectionId& id,
-               unsigned int ssf);
+               const qpid::sys::SecuritySettings& external);
     ~Connection();
     
     ConnectionId getId() const { return self; }
@@ -163,7 +164,7 @@
     void exchange(const std::string& encoded);
 
     void giveReadCredit(int credit);
-    void announce(const std::string& mgmtId, uint32_t ssf);
+    void announce(const std::string& mgmtId, uint32_t ssf, const std::string& authid, bool nodict);
     void abort();
     void deliverClose();
 
@@ -174,7 +175,7 @@
     void managementAgents(const std::string& data);
     void managementSetupState(uint64_t objectNum, uint16_t bootSequence);
 
-    uint32_t getSsf() const { return connectionCtor.ssf; }
+    //uint32_t getSsf() const { return connectionCtor.external.ssf; }
 
   private:
     struct NullFrameHandler : public framing::FrameHandler {
@@ -186,7 +187,7 @@
         sys::ConnectionOutputHandler* out;
         broker::Broker& broker;
         std::string mgmtId;
-        unsigned int ssf;
+        qpid::sys::SecuritySettings external;
         bool isLink;
         uint64_t objectId;
         bool shadow;
@@ -195,17 +196,17 @@
             sys::ConnectionOutputHandler* out_,
             broker::Broker& broker_,
             const std::string& mgmtId_,
-            unsigned int ssf_,
+            const qpid::sys::SecuritySettings& external_,
             bool isLink_=false,
             uint64_t objectId_=0,
             bool shadow_=false
-        ) : out(out_), broker(broker_), mgmtId(mgmtId_), ssf(ssf_),
+        ) : out(out_), broker(broker_), mgmtId(mgmtId_), external(external_),
             isLink(isLink_), objectId(objectId_), shadow(shadow_)
         {}
 
         std::auto_ptr<broker::Connection> construct() {
             return std::auto_ptr<broker::Connection>(
-                new broker::Connection(out, broker, mgmtId, ssf, isLink, objectId, shadow));
+                new broker::Connection(out, broker, mgmtId, external, isLink, objectId, shadow));
         }
     };
 

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.cpp Fri Mar  5 16:51:22 2010
@@ -37,26 +37,26 @@
 
 sys::ConnectionCodec*
 ConnectionCodec::Factory::create(ProtocolVersion v, sys::OutputControl& out, const std::string& id,
-                                 unsigned int ssf) {
+                                 const qpid::sys::SecuritySettings& external) {
     if (v == ProtocolVersion(0, 10))
-        return new ConnectionCodec(v, out, id, cluster, false, false, ssf);
+        return new ConnectionCodec(v, out, id, cluster, false, false, external);
     else if (v == ProtocolVersion(0x80 + 0, 0x80 + 10)) // Catch-up connection
-        return new ConnectionCodec(v, out, id, cluster, true, false, ssf); 
+        return new ConnectionCodec(v, out, id, cluster, true, false, external); 
     return 0;
 }
 
 // Used for outgoing Link connections
 sys::ConnectionCodec*
 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);
+                                 const qpid::sys::SecuritySettings& external) {
+    return new ConnectionCodec(ProtocolVersion(0,10), out, logId, cluster, false, true, external);
 }
 
 ConnectionCodec::ConnectionCodec(
     const ProtocolVersion& v, sys::OutputControl& out,
-    const std::string& logId, Cluster& cluster, bool catchUp, bool isLink, unsigned int ssf
+    const std::string& logId, Cluster& cluster, bool catchUp, bool isLink, const qpid::sys::SecuritySettings& external
 ) : codec(out, logId, isLink),
-    interceptor(new Connection(cluster, codec, logId, cluster.getId(), catchUp, isLink, ssf))
+    interceptor(new Connection(cluster, codec, logId, cluster.getId(), catchUp, isLink, external))
 {
     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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/cluster/ConnectionCodec.h Fri Mar  5 16:51:22 2010
@@ -53,14 +53,14 @@
         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,
-                                     unsigned int conn_ssf);
+                                     const qpid::sys::SecuritySettings& external);
         sys::ConnectionCodec* create(sys::OutputControl&, const std::string& id,
-                                     unsigned int conn_ssf);
+                                     const qpid::sys::SecuritySettings& external);
     };
 
     ConnectionCodec(const framing::ProtocolVersion&, sys::OutputControl& out,
                     const std::string& logId, Cluster& c, bool catchUp, bool isLink,
-                    unsigned int ssf);
+                    const qpid::sys::SecuritySettings& external);
     ~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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/AsynchIOHandler.cpp Fri Mar  5 16:51:22 2010
@@ -22,6 +22,7 @@
 #include "qpid/sys/AsynchIOHandler.h"
 #include "qpid/sys/AsynchIO.h"
 #include "qpid/sys/Socket.h"
+#include "qpid/sys/SecuritySettings.h"
 #include "qpid/framing/AMQP_HighestVersion.h"
 #include "qpid/framing/ProtocolInitiation.h"
 #include "qpid/log/Statement.h"
@@ -144,7 +145,7 @@
             decoded = in.getPosition();
             QPID_LOG(debug, "RECV [" << identifier << "] INIT(" << protocolInit << ")");
             try {
-                codec = factory->create(protocolInit.getVersion(), *this, identifier, 0);
+                codec = factory->create(protocolInit.getVersion(), *this, identifier, SecuritySettings());
                 if (!codec) {
                     //TODO: may still want to revise this...
                     //send valid version header & close connection.
@@ -200,7 +201,7 @@
 
 void AsynchIOHandler::idle(AsynchIO&){
     if (isClient && codec == 0) {
-        codec = factory->create(*this, identifier, 0);
+        codec = factory->create(*this, identifier, SecuritySettings());
         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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ConnectionCodec.h Fri Mar  5 16:51:22 2010
@@ -30,6 +30,7 @@
 
 class InputHandlerFactory;
 class OutputControl;
+struct SecuritySettings;
 
 /**
  * Interface of coder/decoder for a connection of a specific protocol
@@ -49,27 +50,15 @@
     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,
-            unsigned int conn_ssf
+            const SecuritySettings&
         ) = 0;
 
         /** Return "preferred" codec for outbound connections. */
         virtual ConnectionCodec* create(
-            OutputControl&, const std::string& id,
-            unsigned int conn_ssf
+            OutputControl&, const std::string& id, const SecuritySettings&
         ) = 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/RdmaIOPlugin.cpp Fri Mar  5 16:51:22 2010
@@ -27,6 +27,7 @@
 #include "qpid/log/Statement.h"
 #include "qpid/sys/rdma/RdmaIO.h"
 #include "qpid/sys/OutputControl.h"
+#include "qpid/sys/SecuritySettings.h"
 
 #include <boost/bind.hpp>
 #include <boost/lexical_cast.hpp>
@@ -139,7 +140,7 @@
     // but we must be able to send
     assert( codec == 0 );
     assert( aio->writable() && aio->bufferAvailable() );
-    codec = factory->create(*this, identifier, 0);
+    codec = factory->create(*this, identifier, SecuritySettings());
     write(framing::ProtocolInitiation(codec->getVersion()));
 }
 
@@ -186,7 +187,7 @@
         decoded = in.getPosition();
         QPID_LOG(debug, "Rdma: RECV [" << identifier << "] INIT(" << protocolInit << ")");
 
-        codec = factory->create(protocolInit.getVersion(), *this, identifier, 0);
+        codec = factory->create(protocolInit.getVersion(), *this, identifier, SecuritySettings());
 
         // If we failed to create the codec then we don't understand the offered protocol version
         if (!codec) {

Added: qpid/trunk/qpid/cpp/src/qpid/sys/SecuritySettings.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/SecuritySettings.h?rev=919487&view=auto
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/SecuritySettings.h (added)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/SecuritySettings.h Fri Mar  5 16:51:22 2010
@@ -0,0 +1,58 @@
+#ifndef QPID_SYS_SECURITYSETTINGS_H
+#define QPID_SYS_SECURITYSETTINGS_H
+
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+namespace qpid {
+namespace sys {
+
+/**
+ * Conveys security information from a given transport to the upper
+ * layers.
+ */
+struct SecuritySettings
+{
+    /**
+     * Security Strength Factor (SSF). Possible values are:
+     * 
+     *             @li 0  No security
+     *             @li 1  Integrity checking only
+     *             @li >1 Integrity and confidentiality with the number
+     *                    giving the encryption key length.
+     */
+    unsigned int ssf;
+    /**
+     * An authorisation id
+     */
+    std::string authid;
+
+    /**
+     * Disables SASL mechanisms that are vulnerable to passive
+     * dictionary-based password attacks
+     */
+    bool nodict;
+
+    SecuritySettings() : ssf(0), nodict(false) {}
+};
+
+}} // namespace qpid::sys
+
+#endif  /*!QPID_SYS_SECURITYSETTINGS_H*/

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/SslPlugin.cpp
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/SslPlugin.cpp?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/SslPlugin.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/SslPlugin.cpp Fri Mar  5 16:51:22 2010
@@ -41,14 +41,18 @@
 {
     uint16_t port;
     bool clientAuth;
+    bool nodict;
 
     SslServerOptions() : port(5671),
-                         clientAuth(false)
+                         clientAuth(false),
+                         nodict(false)
     {
         addOptions()
             ("ssl-port", optValue(port, "PORT"), "Port on which to listen for SSL connections")
             ("ssl-require-client-authentication", optValue(clientAuth), 
-             "Forces clients to authenticate in order to establish an SSL connection");
+             "Forces clients to authenticate in order to establish an SSL connection")
+            ("ssl-sasl-no-dict", optValue(nodict), 
+             "Disables SASL mechanisms that are vulnerable to passive dictionary-based password attacks");
     }
 };
 
@@ -57,6 +61,7 @@
     qpid::sys::ssl::SslSocket listener;
     const uint16_t listeningPort;
     std::auto_ptr<qpid::sys::ssl::SslAcceptor> acceptor;
+    bool nodict;
 
   public:
     SslProtocolFactory(const SslServerOptions&, int backlog, bool nodelay);
@@ -97,7 +102,8 @@
                     
                     const broker::Broker::Options& opts = broker->getOptions();
                     ProtocolFactory::shared_ptr protocol(new SslProtocolFactory(options,
-                                                                                opts.connectionBacklog, opts.tcpNoDelay));
+                                                                                opts.connectionBacklog,
+                                                                                opts.tcpNoDelay));
                     QPID_LOG(notice, "Listening for SSL connections on TCP port " << protocol->getPort());
                     broker->registerProtocolFactory("ssl", protocol);
                 } catch (const std::exception& e) {
@@ -109,12 +115,13 @@
 } sslPlugin;
 
 SslProtocolFactory::SslProtocolFactory(const SslServerOptions& options, int backlog, bool nodelay) :
-    tcpNoDelay(nodelay), listeningPort(listener.listen(options.port, backlog, options.certName, options.clientAuth))
+    tcpNoDelay(nodelay), listeningPort(listener.listen(options.port, backlog, options.certName, options.clientAuth)),
+    nodict(options.nodict)
 {}
 
 void SslProtocolFactory::established(Poller::shared_ptr poller, const qpid::sys::ssl::SslSocket& s,
                                           ConnectionCodec::Factory* f, bool isClient) {
-    qpid::sys::ssl::SslHandler* async = new qpid::sys::ssl::SslHandler(s.getPeerAddress(), f);
+    qpid::sys::ssl::SslHandler* async = new qpid::sys::ssl::SslHandler(s.getPeerAddress(), f, nodict);
 
     if (tcpNoDelay) {
         s.setTcpNoDelay(tcpNoDelay);

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.cpp Fri Mar  5 16:51:22 2010
@@ -42,13 +42,14 @@
     { delete [] bytes;}
 };
 
-SslHandler::SslHandler(std::string id, ConnectionCodec::Factory* f) :
+SslHandler::SslHandler(std::string id, ConnectionCodec::Factory* f, bool _nodict) :
     identifier(id),
     aio(0),
     factory(f),
     codec(0),
     readError(false),
-    isClient(false)
+    isClient(false),
+    nodict(_nodict)
 {}
 
 SslHandler::~SslHandler() {
@@ -111,7 +112,7 @@
             decoded = in.getPosition();
             QPID_LOG(debug, "RECV [" << identifier << "] INIT(" << protocolInit << ")");
             try {
-                codec = factory->create(protocolInit.getVersion(), *this, identifier, aio->getKeyLen());
+                codec = factory->create(protocolInit.getVersion(), *this, identifier, getSecuritySettings(aio));
                 if (!codec) {
                     //TODO: may still want to revise this...
                     //send valid version header & close connection.
@@ -166,7 +167,7 @@
 
 void SslHandler::idle(SslIO&){
     if (isClient && codec == 0) {
-        codec = factory->create(*this, identifier, aio->getKeyLen());
+        codec = factory->create(*this, identifier, getSecuritySettings(aio));
         write(framing::ProtocolInitiation(codec->getVersion()));
         return;
     }
@@ -183,5 +184,12 @@
         aio->queueWriteClose();
 }
 
+SecuritySettings SslHandler::getSecuritySettings(SslIO* aio)
+{
+    SecuritySettings settings = aio->getSecuritySettings();
+    settings.nodict = nodict;
+    return settings;
+}
+
 
 }}} // namespace qpid::sys::ssl

Modified: qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.h
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.h?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslHandler.h Fri Mar  5 16:51:22 2010
@@ -45,11 +45,13 @@
     ConnectionCodec* codec;
     bool readError;
     bool isClient;
+    bool nodict;
 
     void write(const framing::ProtocolInitiation&);
+    qpid::sys::SecuritySettings getSecuritySettings(SslIO* aio);
 
   public:
-    SslHandler(std::string id, ConnectionCodec::Factory* f);
+    SslHandler(std::string id, ConnectionCodec::Factory* f, bool nodict);
     ~SslHandler();
     void init(SslIO* a, int numBuffs);
 

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.cpp Fri Mar  5 16:51:22 2010
@@ -436,4 +436,9 @@
     }
 }
 
-int SslIO::getKeyLen() {return socket.getKeyLen();}
+SecuritySettings SslIO::getSecuritySettings() {
+    SecuritySettings settings;
+    settings.ssf = socket.getKeyLen();
+    settings.authid = socket.getClientAuthId();
+    return settings;
+}

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslIo.h Fri Mar  5 16:51:22 2010
@@ -22,6 +22,7 @@
  */
 
 #include "qpid/sys/DispatchHandle.h"
+#include "qpid/sys/SecuritySettings.h"
 
 #include <boost/function.hpp>
 #include <deque>
@@ -156,7 +157,7 @@
     bool writeQueueEmpty() { return writeQueue.empty(); }
     BufferBase* getQueuedBuffer();
 
-    int getKeyLen();
+    qpid::sys::SecuritySettings getSecuritySettings();
 
 private:
     ~SslIO();

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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.cpp Fri Mar  5 16:51:22 2010
@@ -102,6 +102,34 @@
     return servName;
 }
 
+const std::string DOMAIN_SEPARATOR("@");
+const std::string DC_SEPARATOR(".");
+const std::string DC("DC");
+const std::string DN_DELIMS(" ,=");
+
+std::string getDomainFromSubject(std::string subject)
+{
+    std::string::size_type last = subject.find_first_not_of(DN_DELIMS, 0);
+    std::string::size_type i = subject.find_first_of(DN_DELIMS, last);
+
+    std::string domain;
+    bool nextTokenIsDC = false;
+    while (std::string::npos != i || std::string::npos != last)
+    {
+        std::string token = subject.substr(last, i - last);
+        if (nextTokenIsDC) {
+            if (domain.size()) domain += DC_SEPARATOR;
+            domain += token;
+            nextTokenIsDC = false;
+        } else if (token == DC) {
+            nextTokenIsDC = true;
+        }
+        last = subject.find_first_not_of(DN_DELIMS, i);
+        i = subject.find_first_of(DN_DELIMS, last);
+    }
+    return domain;
+}
+
 }
 
 SslSocket::SslSocket() : IOHandle(new IOHandlePrivate()), socket(0), prototype(0) 
@@ -294,4 +322,25 @@
     return 0;
 }
 
+std::string SslSocket::getClientAuthId() const
+{
+    std::string authId;
+    CERTCertificate* cert = SSL_PeerCertificate(socket);
+    if (cert) {
+        authId = CERT_GetCommonName(&(cert->subject));
+        /*
+         * The NSS function CERT_GetDomainComponentName only returns
+         * the last component of the domain name, so we have to parse
+         * the subject manually to extract the full domain.
+         */
+        std::string domain = getDomainFromSubject(cert->subjectName);
+        if (!domain.empty()) {
+            authId += DOMAIN_SEPARATOR;
+            authId += domain;
+        }
+        CERT_DestroyCertificate(cert);
+    }
+    return authId;
+}
+
 }}} // 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=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h (original)
+++ qpid/trunk/qpid/cpp/src/qpid/sys/ssl/SslSocket.h Fri Mar  5 16:51:22 2010
@@ -101,6 +101,7 @@
     int getError() const;
 
     int getKeyLen() const;
+    std::string getClientAuthId() const;
 
 private:
     mutable std::string connectname;

Modified: qpid/trunk/qpid/cpp/xml/cluster.xml
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/cpp/xml/cluster.xml?rev=919487&r1=919486&r2=919487&view=diff
==============================================================================
--- qpid/trunk/qpid/cpp/xml/cluster.xml (original)
+++ qpid/trunk/qpid/cpp/xml/cluster.xml Fri Mar  5 16:51:22 2010
@@ -122,6 +122,10 @@
       encryption (e.g. ssl), ssf is the bit length of the key.  Zero if no
       encryption provided. -->
       <field name="ssf" type="uint32"/>
+      <!-- external auth id (e.g. ssl client certificate id) -->
+      <field name="authid" type="str16"/>
+      <!-- exclude certain sasl mechs, used with ssl and sasl-external -->
+      <field name="nodict" type="bit"/>
     </control>
 
     <!-- Marks the cluster-wide point when a connection is considered closed. -->



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