You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ta...@apache.org on 2010/05/03 21:01:41 UTC

svn commit: r940571 - in /activemq/activemq-cpp/trunk/activemq-cpp/src/main: ./ activemq/library/ activemq/transport/tcp/

Author: tabish
Date: Mon May  3 19:01:41 2010
New Revision: 940571

URL: http://svn.apache.org/viewvc?rev=940571&view=rev
Log:
https://issues.apache.org/activemq/browse/AMQCPP-140

Add an initial SSLTransport and its associated Factory instance.  Register it with the library and make a few changes to TcpTransport so it can be extended with other types of streaming Socket transports.

Added:
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp   (with props)
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h   (with props)
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp   (with props)
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h   (with props)
Modified:
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/Makefile.am
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/library/ActiveMQCPP.cpp
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.cpp
    activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.h

Modified: activemq/activemq-cpp/trunk/activemq-cpp/src/main/Makefile.am
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/Makefile.am?rev=940571&r1=940570&r2=940571&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/Makefile.am (original)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/Makefile.am Mon May  3 19:01:41 2010
@@ -128,6 +128,8 @@ cc_sources = \
     activemq/transport/mock/InternalCommandListener.cpp \
     activemq/transport/mock/MockTransport.cpp \
     activemq/transport/mock/MockTransportFactory.cpp \
+    activemq/transport/tcp/SslTransport.cpp \
+    activemq/transport/tcp/SslTransportFactory.cpp \
     activemq/transport/tcp/TcpTransport.cpp \
     activemq/transport/tcp/TcpTransportFactory.cpp \
     activemq/util/ActiveMQProperties.cpp \
@@ -757,6 +759,8 @@ h_sources = \
     activemq/transport/mock/MockTransport.h \
     activemq/transport/mock/MockTransportFactory.h \
     activemq/transport/mock/ResponseBuilder.h \
+    activemq/transport/tcp/SslTransport.h \
+    activemq/transport/tcp/SslTransportFactory.h \
     activemq/transport/tcp/TcpTransport.h \
     activemq/transport/tcp/TcpTransportFactory.h \
     activemq/util/ActiveMQProperties.h \

Modified: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/library/ActiveMQCPP.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/library/ActiveMQCPP.cpp?rev=940571&r1=940570&r2=940571&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/library/ActiveMQCPP.cpp (original)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/library/ActiveMQCPP.cpp Mon May  3 19:01:41 2010
@@ -26,6 +26,7 @@
 
 #include <activemq/transport/mock/MockTransportFactory.h>
 #include <activemq/transport/tcp/TcpTransportFactory.h>
+#include <activemq/transport/tcp/SslTransportFactory.h>
 #include <activemq/transport/failover/FailoverTransportFactory.h>
 
 using namespace activemq;
@@ -81,6 +82,8 @@ void ActiveMQCPP::registerTransports() {
     TransportRegistry::getInstance().registerFactory(
         "tcp", new TcpTransportFactory() );
     TransportRegistry::getInstance().registerFactory(
+        "ssl", new SslTransportFactory() );
+    TransportRegistry::getInstance().registerFactory(
         "mock", new MockTransportFactory() );
     TransportRegistry::getInstance().registerFactory(
         "failover", new FailoverTransportFactory() );

Added: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp?rev=940571&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp (added)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp Mon May  3 19:01:41 2010
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "SslTransport.h"
+
+#include <decaf/lang/Integer.h>
+#include <decaf/lang/Boolean.h>
+#include <decaf/net/ssl/SSLSocket.h>
+#include <decaf/net/ssl/SSLSocketFactory.h>
+#include <decaf/lang/exceptions/NullPointerException.h>
+#include <decaf/lang/exceptions/IllegalArgumentException.h>
+
+#include <memory>
+
+using namespace std;
+using namespace activemq;
+using namespace activemq::io;
+using namespace activemq::transport;
+using namespace activemq::transport::tcp;
+using namespace activemq::exceptions;
+using namespace decaf;
+using namespace decaf::net;
+using namespace decaf::net::ssl;
+using namespace decaf::util;
+using namespace decaf::io;
+using namespace decaf::lang;
+using namespace decaf::lang::exceptions;
+
+////////////////////////////////////////////////////////////////////////////////
+SslTransport::SslTransport( const decaf::net::URI& uri,
+                            const decaf::util::Properties& properties,
+                            const Pointer<Transport>& next ) :
+   TcpTransport( uri, properties, next ) {
+}
+
+////////////////////////////////////////////////////////////////////////////////
+SslTransport::SslTransport( const decaf::util::Properties& properties,
+                            const Pointer<Transport>& next ) :
+    TcpTransport( properties, next ) {
+}
+
+////////////////////////////////////////////////////////////////////////////////
+SslTransport::~SslTransport() {
+}
+
+////////////////////////////////////////////////////////////////////////////////
+Socket* SslTransport::createSocket() {
+
+    try {
+        std::auto_ptr<SocketFactory> factory( SSLSocketFactory::getDefault() );
+        return factory->createSocket();
+    }
+    DECAF_CATCH_RETHROW( IOException )
+    DECAF_CATCH_EXCEPTION_CONVERT( Exception, IOException )
+    DECAF_CATCHALL_THROW( IOException )
+}
+
+////////////////////////////////////////////////////////////////////////////////
+void SslTransport::configureSocket( Socket* socket, decaf::util::Properties& properties ) {
+
+    try{
+
+        if( socket == NULL ) {
+            throw NullPointerException(
+                __FILE__, __LINE__, "Socket instance passed was NULL" );
+        }
+
+        SSLSocket* sslSocket = dynamic_cast<SSLSocket*>( socket );
+        if( sslSocket == NULL ) {
+            throw IllegalArgumentException(
+                __FILE__, __LINE__, "Socket passed was not an SSLSocket instance." );
+        }
+
+        TcpTransport::configureSocket( socket, properties );
+    }
+    DECAF_CATCH_RETHROW( NullPointerException )
+    DECAF_CATCH_RETHROW( IllegalArgumentException )
+    DECAF_CATCH_RETHROW( SocketException )
+    DECAF_CATCH_EXCEPTION_CONVERT( Exception, SocketException )
+    DECAF_CATCHALL_THROW( SocketException )
+}

Propchange: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h?rev=940571&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h (added)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h Mon May  3 19:01:41 2010
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+#ifndef _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORT_H_
+#define _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORT_H_
+
+#include <activemq/util/Config.h>
+
+#include <activemq/transport/tcp/TcpTransport.h>
+
+namespace activemq {
+namespace transport {
+namespace tcp {
+
+    /**
+     * Transport for connecting to a Broker using an SSL Socket.  This transport simply
+     * wraps the TcpTransport and provides the TcpTransport an SSL based Socket pointer
+     * allowing the core TcpTransport logic to be reused.
+     *
+     * @since 3.2.0
+     */
+    class AMQCPP_API SslTransport : public TcpTransport {
+    private:
+
+        SslTransport( const SslTransport& );
+        SslTransport& operator= ( const SslTransport& );
+
+    public:
+
+        /**
+         * Creates a new instance of the SslTransport, the Broker URI is assumed
+         * to be set in the property "transport.uri".
+         *
+         * @param properties the configuration properties for this transport
+         * @param next the next transport in the chain
+         */
+        SslTransport( const decaf::util::Properties& properties,
+                      const Pointer<Transport>& next );
+
+        /**
+         * Creates a new instance of the SslTransport, the uri instance specifies the
+         * host and port to connect to.
+         *
+         * @param uri - The URI containing the host to connect to.
+         * @param properties the configuration properties for this transport
+         * @param next the next transport in the chain
+         */
+        SslTransport( const decaf::net::URI& uri,
+                      const decaf::util::Properties& properties,
+                      const Pointer<Transport>& next );
+
+        virtual ~SslTransport();
+
+    protected:
+
+        /**
+         * {@inheritDoc}
+         */
+        virtual decaf::net::Socket* createSocket();
+
+        /**
+         * {@inheritDoc}
+         */
+        virtual void configureSocket( decaf::net::Socket* socket, decaf::util::Properties& properties );
+
+    };
+
+}}}
+
+#endif /* _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORT_H_ */

Propchange: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransport.h
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp?rev=940571&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp (added)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp Mon May  3 19:01:41 2010
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "SslTransportFactory.h"
+
+#include <activemq/transport/tcp/SslTransport.h>
+
+#include <activemq/transport/IOTransport.h>
+#include <activemq/transport/inactivity/InactivityMonitor.h>
+
+#include <memory>
+
+using namespace std;
+using namespace activemq;
+using namespace activemq::io;
+using namespace activemq::transport;
+using namespace activemq::transport::inactivity;
+using namespace activemq::transport::tcp;
+using namespace activemq::exceptions;
+using namespace decaf;
+using namespace decaf::net;
+using namespace decaf::util;
+using namespace decaf::io;
+using namespace decaf::lang;
+
+////////////////////////////////////////////////////////////////////////////////
+SslTransportFactory::~SslTransportFactory() {
+}
+
+////////////////////////////////////////////////////////////////////////////////
+Pointer<Transport> SslTransportFactory::doCreateComposite( const decaf::net::URI& location,
+                                                           const Pointer<wireformat::WireFormat>& wireFormat,
+                                                           const decaf::util::Properties& properties )
+    throw ( ActiveMQException ) {
+
+    try {
+
+        Pointer<Transport> transport( new SslTransport(
+            location, properties, Pointer<Transport>( new IOTransport( wireFormat ) ) ) );
+
+        if( properties.getProperty( "trnasport.useInactivityMonitor", "true" ) == "true" ) {
+            transport.reset( new InactivityMonitor( transport, properties, wireFormat ) );
+        }
+
+        // If there is a negotiator need then we create and wrap here.
+        if( wireFormat->hasNegotiator() ) {
+            transport = wireFormat->createNegotiator( transport );
+        }
+
+        return transport;
+    }
+    AMQ_CATCH_RETHROW( ActiveMQException )
+    AMQ_CATCH_EXCEPTION_CONVERT( Exception, ActiveMQException )
+    AMQ_CATCHALL_THROW( ActiveMQException )
+}

Propchange: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h?rev=940571&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h (added)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h Mon May  3 19:01:41 2010
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+#ifndef _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORTFACTORY_H_
+#define _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORTFACTORY_H_
+
+#include <activemq/util/Config.h>
+
+#include <activemq/transport/tcp/TcpTransportFactory.h>
+
+namespace activemq {
+namespace transport {
+namespace tcp {
+
+    using decaf::lang::Pointer;
+
+    class SslTransportFactory : public TcpTransportFactory {
+    public:
+
+        virtual ~SslTransportFactory();
+
+    protected:
+
+        /**
+         * Creates a slimed down Transport instance which can be used in composite
+         * transport instances.
+         *
+         * @param location - URI location to connect to.
+         * @param wireFormat - the assigned WireFormat for the new Transport.
+         * @param properties - Properties to apply to the transport.
+         *
+         * @return new Pointer to a SslTransport.
+         * @throws ActiveMQexception if an error occurs
+         */
+        virtual Pointer<Transport> doCreateComposite( const decaf::net::URI& location,
+                                                      const Pointer<wireformat::WireFormat>& wireFormat,
+                                                      const decaf::util::Properties& properties )
+            throw ( exceptions::ActiveMQException );
+
+    };
+
+}}}
+
+#endif /* _ACTIVEMQ_TRANSPORT_TCP_SSLTRANSPORTFACTORY_H_ */

Propchange: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/SslTransportFactory.h
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.cpp?rev=940571&r1=940570&r2=940571&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.cpp (original)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.cpp Mon May  3 19:01:41 2010
@@ -17,13 +17,14 @@
 
 #include "TcpTransport.h"
 
-#include <decaf/lang/Integer.h>
-#include <decaf/net/SocketFactory.h>
 #include <activemq/transport/IOTransport.h>
 #include <activemq/transport/TransportFactory.h>
 
+#include <decaf/lang/exceptions/NullPointerException.h>
+#include <decaf/lang/exceptions/IllegalArgumentException.h>
 #include <decaf/lang/Integer.h>
 #include <decaf/lang/Boolean.h>
+#include <decaf/net/SocketFactory.h>
 
 #include <memory>
 
@@ -38,12 +39,13 @@ using namespace decaf::net;
 using namespace decaf::util;
 using namespace decaf::io;
 using namespace decaf::lang;
+using namespace decaf::lang::exceptions;
 
 ////////////////////////////////////////////////////////////////////////////////
 TcpTransport::TcpTransport( const decaf::net::URI& uri,
                             const decaf::util::Properties& properties,
                             const Pointer<Transport>& next )
-:   TransportFilter( next ) {
+:   TransportFilter( next ), connectTimeout( 0 ), closed( false ) {
 
     this->initialize( uri, properties );
 }
@@ -51,7 +53,7 @@ TcpTransport::TcpTransport( const decaf:
 ////////////////////////////////////////////////////////////////////////////////
 TcpTransport::TcpTransport( const decaf::util::Properties& properties,
                             const Pointer<Transport>& next )
-:   TransportFilter( next ) {
+:   TransportFilter( next ), connectTimeout( 0 ), closed( false ) {
 
     if( !properties.hasProperty( "transport.uri" ) ) {
         throw ActiveMQException(
@@ -100,11 +102,22 @@ void TcpTransport::initialize( const dec
 
     try {
 
-        std::auto_ptr<SocketFactory> factory( SocketFactory::getDefault() );
+        socket.reset( this->createSocket() );
 
-        socket.reset( factory->createSocket() );
+        // Set all Socket Options from the URI options.
+        this->configureSocket( socket.get(), properties );
+
+        // Ensure something is actually passed in for the URI
+        if( uri.getAuthority() == "" ) {
+            throw SocketException( __FILE__, __LINE__,
+                "Connection URI was not provided or is invalid: %s", uri.toString().c_str() );
+        }
+
+        // Connect the socket.
+        string host = uri.getHost();
+        int port = uri.getPort();
 
-        this->configureSocket( *socket, uri, properties );
+        socket->connect( host, port, connectTimeout );
 
         // Cast it to an IO transport so we can wire up the socket
         // input and output streams.
@@ -161,18 +174,21 @@ void TcpTransport::initialize( const dec
 }
 
 ////////////////////////////////////////////////////////////////////////////////
-void TcpTransport::configureSocket( Socket& socket, const URI& uri, const Properties& properties ) {
+Socket* TcpTransport::createSocket() {
 
     try {
+        std::auto_ptr<SocketFactory> factory( SocketFactory::getDefault() );
+        return factory->createSocket();
+    }
+    DECAF_CATCH_RETHROW( IOException )
+    DECAF_CATCH_EXCEPTION_CONVERT( Exception, IOException )
+    DECAF_CATCHALL_THROW( IOException )
+}
 
-        // Ensure something is actually passed in for the URI
-        if( uri.getAuthority() == "" ) {
-            throw SocketException( __FILE__, __LINE__,
-                "SocketTransport::start() - uri not provided" );
-        }
+////////////////////////////////////////////////////////////////////////////////
+void TcpTransport::configureSocket( Socket* socket, const Properties& properties ) {
 
-        string host = uri.getHost();
-        int port = uri.getPort();
+    try {
 
         // Get the linger flag.
         int soLinger = Integer::parseInt(
@@ -195,27 +211,27 @@ void TcpTransport::configureSocket( Sock
             properties.getProperty( "tcpNoDelay", "true" ) );
 
         // Get the socket connect timeout in microseconds. (default to infinite wait).
-        int connectTimeout = Integer::parseInt( properties.getProperty( "soConnectTimeout", "0" ) );
-
-        // Connect the socket.
-        socket.connect( host, port, connectTimeout );
+        this->connectTimeout = Integer::parseInt(
+            properties.getProperty( "soConnectTimeout", "0" ) );
 
         // Set the socket options.
-        socket.setKeepAlive( soKeepAlive );
-        socket.setTcpNoDelay( tcpNoDelay );
+        socket->setKeepAlive( soKeepAlive );
+        socket->setTcpNoDelay( tcpNoDelay );
 
         if( soLinger > 0 ) {
-            socket.setSoLinger( true, soLinger );
+            socket->setSoLinger( true, soLinger );
         }
 
         if( soReceiveBufferSize > 0 ){
-            socket.setReceiveBufferSize( soReceiveBufferSize );
+            socket->setReceiveBufferSize( soReceiveBufferSize );
         }
 
         if( soSendBufferSize > 0 ){
-            socket.setSendBufferSize( soSendBufferSize );
+            socket->setSendBufferSize( soSendBufferSize );
         }
     }
+    DECAF_CATCH_RETHROW( NullPointerException )
+    DECAF_CATCH_RETHROW( IllegalArgumentException )
     DECAF_CATCH_RETHROW( SocketException )
     DECAF_CATCH_EXCEPTION_CONVERT( Exception, SocketException )
     DECAF_CATCHALL_THROW( SocketException )

Modified: activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.h?rev=940571&r1=940570&r2=940571&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.h (original)
+++ activemq/activemq-cpp/trunk/activemq-cpp/src/main/activemq/transport/tcp/TcpTransport.h Mon May  3 19:01:41 2010
@@ -48,6 +48,11 @@ namespace tcp{
     private:
 
         /**
+         * Stores the URI configured Socket connect timeout.
+         */
+        int connectTimeout;
+
+        /**
          * has close been called.
          */
         bool closed;
@@ -75,7 +80,9 @@ namespace tcp{
     public:
 
         /**
-         * Constructor
+         * Creates a new instance of the TcpTransport, the Broker URI is assumed
+         * to be set in the property "transport.uri".
+         *
          * @param properties the configuration properties for this transport
          * @param next the next transport in the chain
          */
@@ -83,7 +90,9 @@ namespace tcp{
                       const Pointer<Transport>& next );
 
         /**
-         * Constructor
+         * Creates a new instance of the TcpTransport, the uri instance specifies the
+         * host and port to connect to.
+         *
          * @param uri - The URI containing the host to connect to.
          * @param properties the configuration properties for this transport
          * @param next the next transport in the chain
@@ -132,14 +141,39 @@ namespace tcp{
             return this->closed;
         }
 
+    protected:
+
+        /**
+         * Create an unconnected Socket instance to be used by the transport to communicate
+         * with the broker.
+         *
+         * @return a newly created unconnected Socket instance.
+         *
+         * @throw IOException if there is an error while creating the unconnected Socket.
+         */
+        virtual decaf::net::Socket* createSocket();
+
+        /**
+         * Using options from configuration URI, configure the socket options before the
+         * Socket instance is connected to the Server.  Subclasses can override this option
+         * to set more configuration options, they should called the base class version to
+         * allow the default set of Socket options to also be configured.
+         *
+         * @param socket
+         *      The Socket instance to configure using options from the given Properties.
+         *
+         * @throw NullPointerException if the Socket instance is null.
+         * @throw IllegalArgumentException if the socket instance is not handled by the class.
+         * @throw SocketException if there is an error while setting one of the Socket options.
+         */
+        virtual void configureSocket( decaf::net::Socket* socket,
+                                      const decaf::util::Properties& properties );
+
     private:
 
         void initialize( const decaf::net::URI& uri,
                          const decaf::util::Properties& properties );
 
-        void configureSocket( decaf::net::Socket& socket,
-                              const decaf::net::URI& uri,
-                              const decaf::util::Properties& properties );
 
     };