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 2009/03/13 15:45:51 UTC

svn commit: r753288 - in /activemq/activemq-cpp/trunk/src: main/ main/activemq/transport/failover/ main/activemq/transport/mock/ main/activemq/wireformat/openwire/ test/ test/activemq/transport/failover/

Author: tabish
Date: Fri Mar 13 14:45:50 2009
New Revision: 753288

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

Updates to the Mock Transport to allow for parameters to be set that control scripted failures for use in testing the Failover Transport.

Added:
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.cpp   (with props)
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.h   (with props)
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/ResponseBuilder.h   (with props)
Modified:
    activemq/activemq-cpp/trunk/src/main/Makefile.am
    activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransport.h
    activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransportListener.cpp
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.cpp
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.h
    activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransportFactory.cpp
    activemq/activemq-cpp/trunk/src/main/activemq/wireformat/openwire/OpenWireResponseBuilder.h
    activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.cpp
    activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.h
    activemq/activemq-cpp/trunk/src/test/testRegistry.cpp

Modified: activemq/activemq-cpp/trunk/src/main/Makefile.am
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/Makefile.am?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/Makefile.am (original)
+++ activemq/activemq-cpp/trunk/src/main/Makefile.am Fri Mar 13 14:45:50 2009
@@ -62,6 +62,7 @@
     activemq/transport/AbstractTransportFactory.cpp \
     activemq/transport/IOTransport.cpp \
     activemq/transport/mock/MockTransport.cpp \
+    activemq/transport/mock/InternalCommandListener.cpp \
     activemq/transport/mock/MockTransportFactory.cpp \
     activemq/transport/correlator/ResponseCorrelator.cpp \
     activemq/transport/tcp/TcpTransport.cpp \
@@ -216,6 +217,8 @@
     activemq/transport/failover/FailoverTransportListener.h \
     activemq/transport/mock/MockTransport.h \
     activemq/transport/mock/MockTransportFactory.h \
+    activemq/transport/mock/InternalCommandListener.h \
+    activemq/transport/mock/ResponseBuilder.h \
     activemq/transport/correlator/FutureResponse.h \
     activemq/transport/correlator/ResponseCorrelator.h \
     activemq/transport/tcp/TcpTransport.h \

Modified: activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransport.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransport.h?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransport.h (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransport.h Fri Mar 13 14:45:50 2009
@@ -237,6 +237,22 @@
         }
 
         /**
+         * Returns true if the Transport has been initialized by a BrokerInfo command.
+         * @return true if the Transport has been initialized by a BrokerInfo command.
+         */
+        bool isInitialized() const {
+            return this->initialized;
+        }
+
+        /**
+         * Sets the initialized state of this Transport to true.
+         * @param value - true if this Transport has been initialized.
+         */
+        void setInitialized( bool value ) {
+            this->initialized = value;
+        }
+
+        /**
          * Narrows down a Chain of Transports to a specific Transport to allow a
          * higher level transport to skip intermediate Transports in certain
          * circumstances.

Modified: activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransportListener.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransportListener.cpp?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransportListener.cpp (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/failover/FailoverTransportListener.cpp Fri Mar 13 14:45:50 2009
@@ -71,7 +71,7 @@
         }
     }
 
-    if( !parent->initialized && command->isBrokerInfo() ) {
+    if( !parent->isInitialized() && command->isBrokerInfo() ) {
 
         Pointer<BrokerInfo> info = command.dynamicCast<BrokerInfo>();
         std::vector< Pointer<BrokerInfo> >& peers = info->getPeerBrokerInfos();
@@ -79,7 +79,7 @@
             std::string brokerString = peers[i]->getBrokerURL();
             parent->add( brokerString );
         }
-        parent->initialized = true;
+        parent->setInitialized( true );
     }
 
     if( parent->transportListener != NULL ) {

Added: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.cpp?rev=753288&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.cpp (added)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.cpp Fri Mar 13 14:45:50 2009
@@ -0,0 +1,100 @@
+/*
+ * 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 "InternalCommandListener.h"
+
+#include <activemq/transport/mock/MockTransport.h>
+
+using namespace activemq;
+using namespace activemq::transport;
+using namespace activemq::transport::mock;
+using namespace activemq::exceptions;
+using namespace activemq::wireformat;
+using namespace decaf::lang;
+using namespace decaf::lang::exceptions;
+
+////////////////////////////////////////////////////////////////////////////////
+InternalCommandListener::InternalCommandListener() : startedLatch(1) {
+    transport = NULL;
+    done = false;
+
+    this->start();
+    startedLatch.await();
+}
+
+////////////////////////////////////////////////////////////////////////////////
+InternalCommandListener::~InternalCommandListener() {
+    done = true;
+    synchronized( &inboundQueue ) {
+        inboundQueue.notifyAll();
+    }
+    this->join();
+
+    inboundQueue.clear();
+}
+
+////////////////////////////////////////////////////////////////////////////////
+void InternalCommandListener::onCommand( const Pointer<Command>& command ) {
+    synchronized( &inboundQueue ) {
+        // Create a response now before the caller has a
+        // chance to destroy the command.
+        responseBuilder->buildIncomingCommands( command, inboundQueue );
+
+        // Wake up the thread, messages are dispatched from there.
+        inboundQueue.notifyAll();
+    }
+}
+
+////////////////////////////////////////////////////////////////////////////////
+void InternalCommandListener::run() {
+
+    try {
+
+        synchronized( &inboundQueue ) {
+
+            while( !done ) {
+                startedLatch.countDown();
+
+                while( inboundQueue.empty() && !done ){
+                    inboundQueue.wait();
+                }
+
+                if( done || transport == NULL ) {
+                    continue;
+                }
+
+                // If we created a response then send it.
+                while( !inboundQueue.empty() ) {
+
+                    Pointer<Command> command = inboundQueue.pop();
+
+                    if( command->isMessage() && transport->isFailOnReceiveMessage() ) {
+                        transport->setNumReceivedMessages( transport->getNumReceivedMessages() + 1 );
+
+                        if( transport->getNumReceivedMessages() > transport->getNumReceivedMessageBeforeFail() ) {
+                            transport->fireException(
+                                CommandIOException( __FILE__, __LINE__, "Failed to Send Message.") );;
+                        }
+                    }
+
+                    transport->fireCommand( command );
+                }
+            }
+        }
+    }
+    AMQ_CATCHALL_NOTHROW()
+}

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

Added: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.h?rev=753288&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.h (added)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/InternalCommandListener.h Fri Mar 13 14:45:50 2009
@@ -0,0 +1,77 @@
+/*
+ * 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_MOCK_INTERNALCOMMANDLISTENER_H_
+#define _ACTIVEMQ_TRANSPORT_MOCK_INTERNALCOMMANDLISTENER_H_
+
+#include <activemq/util/Config.h>
+#include <activemq/transport/mock/ResponseBuilder.h>
+#include <activemq/transport/DefaultTransportListener.h>
+
+#include <decaf/lang/Thread.h>
+#include <decaf/lang/Pointer.h>
+#include <decaf/util/StlQueue.h>
+#include <decaf/util/concurrent/Concurrent.h>
+#include <decaf/util/concurrent/atomic/AtomicInteger.h>
+#include <decaf/util/concurrent/CountDownLatch.h>
+
+namespace activemq {
+namespace transport {
+namespace mock {
+
+    class MockTransport;
+
+    /**
+     * Listens for Commands sent from the MockTransport.  This class
+     * processes all outbound commands and sends responses that are
+     * constructed by calling the Protocol provided ResponseBuilder
+     * and getting a set of Commands to send back into the MockTransport
+     * as incoming Commands and Responses.
+     */
+    class AMQCPP_API InternalCommandListener : public DefaultTransportListener,
+                                               public decaf::lang::Thread {
+    private:
+
+        MockTransport* transport;
+        Pointer<ResponseBuilder> responseBuilder;
+        bool done;
+        decaf::util::concurrent::CountDownLatch startedLatch;
+        decaf::util::StlQueue< Pointer<Command> > inboundQueue;
+
+    public:
+
+        InternalCommandListener();
+
+        virtual ~InternalCommandListener();
+
+        void setTransport( MockTransport* transport ){
+            this->transport = transport;
+        }
+
+        void setResponseBuilder( const Pointer<ResponseBuilder>& responseBuilder ) {
+            this->responseBuilder = responseBuilder;
+        }
+
+        virtual void onCommand( const Pointer<Command>& command );
+
+        void run();
+
+    };
+
+}}}
+
+#endif /* _ACTIVEMQ_TRANSPORT_MOCK_INTERNALCOMMANDLISTENER_H_ */

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

Modified: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.cpp?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.cpp (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.cpp Fri Mar 13 14:45:50 2009
@@ -40,6 +40,14 @@
     this->nextCommandId.set( 0 );
     this->instance = this;
 
+    // Script Properties.
+    this->failOnSendMessage = false;
+    this->numSentMessageBeforeFail = 0;
+    this->numSentMessages = 0;
+    this->failOnReceiveMessage = false;
+    this->numReceivedMessageBeforeFail = 0;
+    this->numReceivedMessages = 0;
+
     // Configure the Internal Listener this is the Fake Broker.
     this->internalListener.setTransport( this );
     this->internalListener.setResponseBuilder( responseBuilder );
@@ -52,6 +60,15 @@
 
     try{
 
+        if( command->isMessage() && this->failOnSendMessage ) {
+            this->numSentMessages++;
+
+            if( this->numSentMessages > this->numSentMessageBeforeFail ) {
+                throw CommandIOException(
+                    __FILE__, __LINE__, "Failed to Send Message.");
+            }
+        }
+
         // Process and send any new Commands back.
         internalListener.onCommand( command );
 
@@ -77,6 +94,15 @@
 
         if( responseBuilder != NULL ){
 
+            if( command->isMessage() && this->failOnSendMessage ) {
+                this->numSentMessages++;
+
+                if( this->numSentMessages > this->numSentMessageBeforeFail ) {
+                    throw CommandIOException(
+                        __FILE__, __LINE__, "Failed to Send Message.");
+                }
+            }
+
             // Notify external Client of command that we "sent"
             if( outgoingListener != NULL ){
                 outgoingListener->onCommand( command );

Modified: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.h?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.h (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransport.h Fri Mar 13 14:45:50 2009
@@ -24,9 +24,12 @@
 #include <activemq/transport/TransportListener.h>
 #include <activemq/transport/DefaultTransportListener.h>
 #include <activemq/transport/CommandIOException.h>
+#include <activemq/transport/mock/ResponseBuilder.h>
+#include <activemq/transport/mock/InternalCommandListener.h>
 #include <activemq/wireformat/WireFormat.h>
 
 #include <decaf/lang/Thread.h>
+#include <decaf/lang/Pointer.h>
 #include <decaf/util/StlQueue.h>
 #include <decaf/util/concurrent/Concurrent.h>
 #include <decaf/util/concurrent/atomic/AtomicInteger.h>
@@ -58,124 +61,6 @@
      * messages that might result from a message sent to the Broker.
      */
     class AMQCPP_API MockTransport : public Transport{
-    public:
-
-        /**
-         * Interface for all Protocols to implement that defines the behavior
-         * of the Broker in response to messages of that protocol.
-         */
-        class AMQCPP_API ResponseBuilder {
-        public:
-
-            virtual ~ResponseBuilder() {}
-
-            /**
-             * Given a Command, check if it requires a response and return the
-             * appropriate Response that the Broker would send for this Command
-             * @param command - The command to build a response for
-             * @return A Response object pointer, or NULL if no response.
-             */
-            virtual Pointer<Response> buildResponse( const Pointer<Command>& command ) = 0;
-
-            /**
-             * When called the ResponseBuilder must construct all the
-             * Responses or Asynchronous commands that would be sent to
-             * this client by the Broker upon receipt of the passed command.
-             * @param command - The Command being sent to the Broker.
-             * @param queue - Queue of Command sent back from the broker.
-             */
-            virtual void buildIncomingCommands(
-                const Pointer<Command>& command,
-                decaf::util::StlQueue< Pointer<Command> >& queue ) = 0;
-
-        };
-
-        /**
-         * Listens for Commands sent from the MockTransport.  This class
-         * processes all outbound commands and sends responses that are
-         * constructed by calling the Protocol provided ResponseBuilder
-         * and getting a set of Commands to send back into the MockTransport
-         * as incoming Commands and Responses.
-         */
-        class InternalCommandListener :
-            public DefaultTransportListener,
-            public decaf::lang::Thread {
-
-        private:
-
-            MockTransport* transport;
-            Pointer<ResponseBuilder> responseBuilder;
-            bool done;
-            decaf::util::concurrent::CountDownLatch startedLatch;
-            decaf::util::StlQueue< Pointer<Command> > inboundQueue;
-
-        public:
-
-            InternalCommandListener() : startedLatch(1) {
-                transport = NULL;
-                done = false;
-
-                this->start();
-                startedLatch.await();
-            }
-
-            virtual ~InternalCommandListener() {
-                done = true;
-                synchronized( &inboundQueue ) {
-                    inboundQueue.notifyAll();
-                }
-                this->join();
-
-                inboundQueue.clear();
-            }
-
-            void setTransport( MockTransport* transport ){
-                this->transport = transport;
-            }
-
-            void setResponseBuilder( const Pointer<ResponseBuilder>& responseBuilder ) {
-                this->responseBuilder = responseBuilder;
-            }
-
-            virtual void onCommand( const Pointer<Command>& command ) {
-                synchronized( &inboundQueue )
-                {
-                    // Create a response now before the caller has a
-                    // chance to destroy the command.
-                    responseBuilder->buildIncomingCommands( command, inboundQueue );
-
-                    // Wake up the thread, messages are dispatched from there.
-                    inboundQueue.notifyAll();
-                }
-            }
-
-            void run() {
-                try {
-
-                    synchronized( &inboundQueue ) {
-
-                        while( !done ) {
-                            startedLatch.countDown();
-
-                            while( inboundQueue.empty() && !done ){
-                                inboundQueue.wait();
-                            }
-
-                            if( done || transport == NULL ) {
-                                continue;
-                            }
-
-                            // If we created a response then send it.
-                            while( !inboundQueue.empty() ) {
-                                transport->fireCommand( inboundQueue.pop() );
-                            }
-                        }
-                    }
-                }
-                AMQ_CATCHALL_NOTHROW()
-            }
-        };
-
     private:
 
         Pointer<ResponseBuilder> responseBuilder;
@@ -186,6 +71,13 @@
         InternalCommandListener internalListener;
         static MockTransport* instance;
 
+        bool failOnSendMessage;
+        int numSentMessageBeforeFail;
+        int numSentMessages;
+        bool failOnReceiveMessage;
+        int numReceivedMessageBeforeFail;
+        int numReceivedMessages;
+
     public:
 
         MockTransport( const Pointer<wireformat::WireFormat>& wireFormat,
@@ -322,9 +214,59 @@
          * @param uri
          * @throws IOException on failure of if not supported
          */
-        virtual void reconnect( const decaf::net::URI& uri )
+        virtual void reconnect( const decaf::net::URI& uri AMQCPP_UNUSED )
             throw( decaf::io::IOException ) {}
 
+    public:  // Property Getters and Setters
+
+        bool isFailOnSendMessage() const {
+            return this->failOnSendMessage;
+        }
+
+        void setFailOnSendMessage( bool value ) {
+            this->failOnSendMessage = value;
+        }
+
+        int getNumSentMessageBeforeFail() const {
+            return this->numSentMessageBeforeFail;
+        }
+
+        void setNumSentMessageBeforeFail( int value ) {
+            this->numSentMessageBeforeFail = value;
+        }
+
+        int getNumSentMessages() const {
+            return this->numSentMessages;
+        }
+
+        void setNumSentMessages( int value ) {
+            this->numSentMessages = value;
+        }
+
+        bool isFailOnReceiveMessage() const {
+            return this->failOnReceiveMessage;
+        }
+
+        void setFailOnReceiveMessage( bool value ) {
+            this->failOnReceiveMessage = value;
+        }
+
+        int getNumReceivedMessageBeforeFail() const {
+            return this->numReceivedMessageBeforeFail;
+        }
+
+        void setNumReceivedMessageBeforeFail( int value ) {
+            this->numReceivedMessageBeforeFail = value;
+        }
+
+        int getNumReceivedMessages() const {
+            return this->numReceivedMessages;
+        }
+
+        void setNumReceivedMessages( int value ) {
+            this->numReceivedMessages = value;
+        }
+
     };
 
 }}}

Modified: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransportFactory.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransportFactory.cpp?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransportFactory.cpp (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/MockTransportFactory.cpp Fri Mar 13 14:45:50 2009
@@ -22,8 +22,12 @@
 #include <activemq/wireformat/openwire/OpenWireResponseBuilder.h>
 #include <activemq/transport/Transport.h>
 #include <activemq/transport/mock/MockTransport.h>
+#include <activemq/transport/mock/ResponseBuilder.h>
 #include <activemq/util/URISupport.h>
 
+#include <decaf/lang/Boolean.h>
+#include <decaf/io/IOException.h>
+
 using namespace activemq;
 using namespace activemq::util;
 using namespace activemq::wireformat;
@@ -34,6 +38,7 @@
 using namespace activemq::exceptions;
 using namespace decaf;
 using namespace decaf::util;
+using namespace decaf::io;
 using namespace decaf::lang;
 
 ////////////////////////////////////////////////////////////////////////////////
@@ -97,7 +102,12 @@
         std::string wireFormatName =
             properties.getProperty( "wireFormat", "openwire" );
 
-        Pointer<MockTransport::ResponseBuilder> builder;
+        if( properties.getProperty( "failOnCreate", "false" ) == "true" ) {
+            throw IOException(
+                __FILE__, __LINE__, "Failed to Create MockTransport." );
+        }
+
+        Pointer<ResponseBuilder> builder;
 
         if( wireFormatName == "stomp" ) {
 //            builder.reset( new wireformat::stomp::StompResponseBuilder() );
@@ -105,7 +115,11 @@
             builder.reset( new wireformat::openwire::OpenWireResponseBuilder() );
         }
 
-        return Pointer<Transport>( new MockTransport( wireFormat, builder ) );
+        Pointer<Transport> transport( new MockTransport( wireFormat, builder ) );
+
+
+
+        return transport;
     }
     AMQ_CATCH_RETHROW( ActiveMQException )
     AMQ_CATCH_EXCEPTION_CONVERT( Exception, ActiveMQException )

Added: activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/ResponseBuilder.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/ResponseBuilder.h?rev=753288&view=auto
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/ResponseBuilder.h (added)
+++ activemq/activemq-cpp/trunk/src/main/activemq/transport/mock/ResponseBuilder.h Fri Mar 13 14:45:50 2009
@@ -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.
+ */
+
+#ifndef _ACTIVEMQ_TRANSPORT_MOCK_RESPONSEBUILDER_H_
+#define _ACTIVEMQ_TRANSPORT_MOCK_RESPONSEBUILDER_H_
+
+#include <activemq/util/Config.h>
+
+#include <activemq/commands/Command.h>
+#include <activemq/commands/Response.h>
+
+#include <decaf/lang/Pointer.h>
+#include <decaf/util/StlQueue.h>
+
+namespace activemq {
+namespace transport {
+namespace mock {
+
+    using decaf::lang::Pointer;
+    using activemq::commands::Command;
+    using activemq::commands::Response;
+
+    /**
+     * Interface for all Protocols to implement that defines the behavior
+     * of the Broker in response to messages of that protocol.
+     */
+    class AMQCPP_API ResponseBuilder {
+    public:
+
+        virtual ~ResponseBuilder() {}
+
+        /**
+         * Given a Command, check if it requires a response and return the
+         * appropriate Response that the Broker would send for this Command
+         * @param command - The command to build a response for
+         * @return A Response object pointer, or NULL if no response.
+         */
+        virtual Pointer<Response> buildResponse( const Pointer<Command>& command ) = 0;
+
+        /**
+         * When called the ResponseBuilder must construct all the
+         * Responses or Asynchronous commands that would be sent to
+         * this client by the Broker upon receipt of the passed command.
+         * @param command - The Command being sent to the Broker.
+         * @param queue - Queue of Command sent back from the broker.
+         */
+        virtual void buildIncomingCommands(
+            const Pointer<Command>& command,
+            decaf::util::StlQueue< Pointer<Command> >& queue ) = 0;
+
+    };
+
+}}}
+
+#endif /* _ACTIVEMQ_TRANSPORT_MOCK_RESPONSEBUILDER_H_ */

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

Modified: activemq/activemq-cpp/trunk/src/main/activemq/wireformat/openwire/OpenWireResponseBuilder.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/main/activemq/wireformat/openwire/OpenWireResponseBuilder.h?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/main/activemq/wireformat/openwire/OpenWireResponseBuilder.h (original)
+++ activemq/activemq-cpp/trunk/src/main/activemq/wireformat/openwire/OpenWireResponseBuilder.h Fri Mar 13 14:45:50 2009
@@ -19,7 +19,7 @@
 #define ACTIVEMQ_WIREFORMAT_OPENWIRE_OPENWIRERESPONSEBUILDER_H_
 
 #include <activemq/util/Config.h>
-#include <activemq/transport/mock/MockTransport.h>
+#include <activemq/transport/mock/ResponseBuilder.h>
 #include <decaf/util/StlQueue.h>
 #include <decaf/lang/Pointer.h>
 
@@ -30,7 +30,7 @@
     using decaf::lang::Pointer;
 
     class AMQCPP_API OpenWireResponseBuilder :
-        public transport::mock::MockTransport::ResponseBuilder{
+        public transport::mock::ResponseBuilder{
     public:
 
         OpenWireResponseBuilder() {}

Modified: activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.cpp?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.cpp (original)
+++ activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.cpp Fri Mar 13 14:45:50 2009
@@ -21,11 +21,13 @@
 #include <activemq/transport/failover/FailoverTransport.h>
 #include <activemq/exceptions/ActiveMQException.h>
 #include <decaf/lang/Pointer.h>
+#include <decaf/lang/Thread.h>
 
 using namespace activemq;
 using namespace activemq::transport;
 using namespace activemq::transport::failover;
 using namespace activemq::exceptions;
+using namespace decaf::lang;
 using namespace decaf::util;
 
 ////////////////////////////////////////////////////////////////////////////////
@@ -58,3 +60,27 @@
     transport->close();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+void FailoverTransportTest::testTransportCreateWithBackups() {
+
+    std::string uri = "failover://(mock://localhost:61616,mock://localhost:61618)?randomize=false&backup=true";
+
+    DefaultTransportListener listener;
+    FailoverTransportFactory factory;
+
+    Pointer<Transport> transport( factory.create( uri ) );
+    CPPUNIT_ASSERT( transport != NULL );
+    transport->setTransportListener( &listener );
+
+    FailoverTransport* failover = dynamic_cast<FailoverTransport*>(
+        transport->narrow( typeid( FailoverTransport ) ) );
+
+    CPPUNIT_ASSERT( failover != NULL );
+    CPPUNIT_ASSERT( failover->isRandomize() == false );
+    CPPUNIT_ASSERT( failover->isBackup() == true );
+
+    Thread::sleep( 1000 );
+
+    transport->start();
+    transport->close();
+}

Modified: activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.h
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.h?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.h (original)
+++ activemq/activemq-cpp/trunk/src/test/activemq/transport/failover/FailoverTransportTest.h Fri Mar 13 14:45:50 2009
@@ -30,6 +30,7 @@
 
         CPPUNIT_TEST_SUITE( FailoverTransportTest );
         CPPUNIT_TEST( testTransportCreate );
+        CPPUNIT_TEST( testTransportCreateWithBackups );
         CPPUNIT_TEST_SUITE_END();
 
     public:
@@ -38,6 +39,7 @@
         virtual ~FailoverTransportTest();
 
         void testTransportCreate();
+        void testTransportCreateWithBackups();
 
     };
 

Modified: activemq/activemq-cpp/trunk/src/test/testRegistry.cpp
URL: http://svn.apache.org/viewvc/activemq/activemq-cpp/trunk/src/test/testRegistry.cpp?rev=753288&r1=753287&r2=753288&view=diff
==============================================================================
--- activemq/activemq-cpp/trunk/src/test/testRegistry.cpp (original)
+++ activemq/activemq-cpp/trunk/src/test/testRegistry.cpp Fri Mar 13 14:45:50 2009
@@ -75,10 +75,10 @@
 //CPPUNIT_TEST_SUITE_REGISTRATION( activemq::core::ActiveMQConnectionTest );
 //#include <activemq/core/ActiveMQSessionTest.h>
 //CPPUNIT_TEST_SUITE_REGISTRATION( activemq::core::ActiveMQSessionTest );
-
+//
 #include <activemq/transport/failover/FailoverTransportTest.h>
 CPPUNIT_TEST_SUITE_REGISTRATION( activemq::transport::failover::FailoverTransportTest );
-
+//
 //#include <activemq/transport/correlator/ResponseCorrelatorTest.h>
 //CPPUNIT_TEST_SUITE_REGISTRATION( activemq::transport::correlator::ResponseCorrelatorTest );
 //