You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2010/03/10 11:14:27 UTC

svn commit: r921281 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/component/mock/ camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/model/loadbalancer/ camel-core/src/main/java/org/apache/camel/pr...

Author: davsclaus
Date: Wed Mar 10 10:14:27 2010
New Revision: 921281

URL: http://svn.apache.org/viewvc?rev=921281&view=rev
Log:
CAMEL-2534: Improved failover LB with round robin, max failover attempts and inherit error handler options.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverMaximumFailoverAttemptsTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java   (contents, props changed)
      - copied, changed from r921228, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringAggregatorTest.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml   (contents, props changed)
      - copied, changed from r921228, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafter.xml
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/LoadBalancerSupport.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/ContextTestSupport.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java Wed Mar 10 10:14:27 2010
@@ -151,6 +151,22 @@ public class MockEndpoint extends Defaul
         }
     }
 
+    /**
+     * Reset all mock endpoints
+     *
+     * @param context the camel context used to find all the available endpoints to reset
+     */
+    public static void resetMocks(CamelContext context) {
+        ObjectHelper.notNull(context, "camelContext");
+        Collection<Endpoint> endpoints = context.getEndpoints();
+        for (Endpoint endpoint : endpoints) {
+            if (endpoint instanceof MockEndpoint) {
+                MockEndpoint mockEndpoint = (MockEndpoint) endpoint;
+                mockEndpoint.reset();
+            }
+        }
+    }
+
     public static void expectsMessageCount(int count, MockEndpoint... endpoints) throws InterruptedException {
         for (MockEndpoint endpoint : endpoints) {
             endpoint.setExpectedMessageCount(count);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java Wed Mar 10 10:14:27 2010
@@ -20,7 +20,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -123,8 +122,7 @@ public class LoadBalanceDefinition exten
             processor = wrapProcessor(routeContext, processor);
             loadBalancer.addProcessor(processor);
         }
-
-        return loadBalancer;
+       return loadBalancer;
     }
     
     // Fluent API
@@ -143,22 +141,45 @@ public class LoadBalanceDefinition exten
     
     /**
      * Uses fail over load balancer
-     * 
+     * <p/>
+     * Will not round robin and inherit the error handler.
+     *
      * @return the builder
      */
     public LoadBalanceDefinition failover() {
-        loadBalancerType = new LoadBalancerDefinition(new FailOverLoadBalancer());
-        return this;
+        return failover(-1, true, false);
     }
     
     /**
      * Uses fail over load balancer
-     * 
+     * <p/>
+     * Will not round robin and inherit the error handler.
+     *
      * @param exceptions exception classes which we want to failover if one of them was thrown
      * @return the builder
      */
     public LoadBalanceDefinition failover(Class<?>... exceptions) {
-        loadBalancerType = new LoadBalancerDefinition(new FailOverLoadBalancer(Arrays.asList(exceptions)));
+        return failover(-1, true, false, exceptions);
+    }
+
+    /**
+     * Uses fail over load balancer
+     *
+     * @param maximumFailoverAttempts  maximum number of failover attempts before exhausting.
+     *                                 Use -1 to newer exhaust when round robin is also enabled.
+     *                                 If round robin is disabled then it will exhaust when there are no more endpoints to failover
+     * @param inheritErrorHandler      whether or not to inherit error handler.
+     *                                 If <tt>false</tt> then it will failover immediately in case of an exception
+     * @param roundRobin               whether or not to use round robin (which keeps state)
+     * @param exceptions               exception classes which we want to failover if one of them was thrown
+     * @return the builder
+     */
+    public LoadBalanceDefinition failover(int maximumFailoverAttempts, boolean inheritErrorHandler, boolean roundRobin, Class<?>... exceptions) {
+        FailOverLoadBalancer failover = new FailOverLoadBalancer(Arrays.asList(exceptions));
+        failover.setMaximumFailoverAttempts(maximumFailoverAttempts);
+        failover.setRoundRobin(roundRobin);
+        loadBalancerType = new LoadBalancerDefinition(failover);
+        this.setInheritErrorHandler(inheritErrorHandler);
         return this;
     }
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java Wed Mar 10 10:14:27 2010
@@ -20,6 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
@@ -34,9 +35,15 @@ public class FailoverLoadBalancerDefinit
 
     @XmlElement(name = "exception")
     private List<String> exceptions = new ArrayList<String>();
+    @XmlAttribute
+    private Boolean roundRobin = Boolean.FALSE;
+    @XmlAttribute
+    private Integer maximumFailoverAttempts;
 
     @Override
     protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
+        FailOverLoadBalancer answer;
+
         if (!exceptions.isEmpty()) {
             List<Class<?>> classes = new ArrayList<Class<?>>();
             for (String name : exceptions) {
@@ -46,10 +53,19 @@ public class FailoverLoadBalancerDefinit
                 }
                 classes.add(type);
             }
-            return new FailOverLoadBalancer(classes);
+            answer = new FailOverLoadBalancer(classes);
         } else {
-            return new FailOverLoadBalancer();
+            answer = new FailOverLoadBalancer();
+        }
+
+        if (getMaximumFailoverAttempts() != null) {
+            answer.setMaximumFailoverAttempts(getMaximumFailoverAttempts());
+        }
+        if (isRoundRobin() != null) {
+            answer.setRoundRobin(isRoundRobin());
         }
+
+        return answer;
     }
 
     public List<String> getExceptions() {
@@ -60,6 +76,22 @@ public class FailoverLoadBalancerDefinit
         this.exceptions = exceptions;
     }
 
+    public Boolean isRoundRobin() {
+        return roundRobin;
+    }
+
+    public void setRoundRobin(Boolean roundRobin) {
+        this.roundRobin = roundRobin;
+    }
+
+    public Integer getMaximumFailoverAttempts() {
+        return maximumFailoverAttempts;
+    }
+
+    public void setMaximumFailoverAttempts(Integer maximumFailoverAttempts) {
+        this.maximumFailoverAttempts = maximumFailoverAttempts;
+    }
+
     @Override
     public String toString() {
         return "FailoverLoadBalancer";

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/FailOverLoadBalancer.java Wed Mar 10 10:14:27 2010
@@ -23,11 +23,16 @@ import org.apache.camel.Processor;
 import org.apache.camel.util.ObjectHelper;
 
 /**
- * This FailOverLoadBalancer will failover to use next processor when an exception occured
+ * This FailOverLoadBalancer will failover to use next processor when an exception occurred
  */
 public class FailOverLoadBalancer extends LoadBalancerSupport {
 
     private final List<Class<?>> exceptions;
+    private boolean roundRobin;
+    private int maximumFailoverAttempts = -1;
+
+    // stateful counter
+    private int counter = -1;
 
     public FailOverLoadBalancer() {
         this.exceptions = null;
@@ -35,6 +40,7 @@ public class FailOverLoadBalancer extend
 
     public FailOverLoadBalancer(List<Class<?>> exceptions) {
         this.exceptions = exceptions;
+
         for (Class<?> type : exceptions) {
             if (!ObjectHelper.isAssignableFrom(Throwable.class, type)) {
                 throw new IllegalArgumentException("Class is not an instance of Throwable: " + type);
@@ -46,6 +52,22 @@ public class FailOverLoadBalancer extend
         return exceptions;
     }
 
+    public boolean isRoundRobin() {
+        return roundRobin;
+    }
+
+    public void setRoundRobin(boolean roundRobin) {
+        this.roundRobin = roundRobin;
+    }
+
+    public int getMaximumFailoverAttempts() {
+        return maximumFailoverAttempts;
+    }
+
+    public void setMaximumFailoverAttempts(int maximumFailoverAttempts) {
+        this.maximumFailoverAttempts = maximumFailoverAttempts;
+    }
+
     /**
      * Should the given failed Exchange failover?
      *
@@ -78,22 +100,53 @@ public class FailOverLoadBalancer extend
         }
 
         int index = 0;
+        int attempts = 0;
+
+        // pick the first endpoint to use
+        if (isRoundRobin()) {
+            if (++counter >= list.size()) {
+                counter = 0;
+            }
+            index = counter;
+        }
+        if (log.isDebugEnabled()) {
+            log.debug("Failover starting with endpoint index " + index);
+        }
+
         Processor processor = list.get(index);
 
         // process the first time
-        processExchange(processor, exchange);
+        processExchange(processor, exchange, attempts);
 
         // loop while we should fail over
         while (shouldFailOver(exchange)) {
+            attempts++;
+            // are we exhausted by attempts?
+            if (maximumFailoverAttempts > -1 && attempts > maximumFailoverAttempts) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Braking out of failover after " + attempts + " failover attempts");
+                }
+                break;
+            }
+
             index++;
+            counter++;
+
             if (index < list.size()) {
                 // try again but prepare exchange before we failover
                 prepareExchangeForFailover(exchange);
                 processor = list.get(index);
-                processExchange(processor, exchange);
+                processExchange(processor, exchange, attempts);
             } else {
-                // no more processors to try
-                break;
+                if (isRoundRobin()) {
+                    log.debug("Failover is round robin enabled and therefore starting from the first endpoint");
+                    index = 0;
+                    counter = 0;
+                } else {
+                    // no more processors to try
+                    log.debug("Braking out of failover as we reach the end of endpoints to use for failover");
+                    break;
+                }
             }
         }
     }
@@ -113,11 +166,14 @@ public class FailOverLoadBalancer extend
         exchange.getIn().removeHeader(Exchange.REDELIVERY_COUNTER);
     }
 
-    private void processExchange(Processor processor, Exchange exchange) {
+    private void processExchange(Processor processor, Exchange exchange, int attempt) {
         if (processor == null) {
             throw new IllegalStateException("No processors could be chosen to process " + exchange);
         }
         try {
+            if (log.isDebugEnabled()) {
+                log.debug("Processing failover at attempt " + attempt + " for exchange: " + exchange);
+            }
             processor.process(exchange);
         } catch (Exception e) {
             exchange.setException(e);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/LoadBalancerSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/LoadBalancerSupport.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/LoadBalancerSupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/LoadBalancerSupport.java Wed Mar 10 10:14:27 2010
@@ -24,6 +24,8 @@ import org.apache.camel.Navigate;
 import org.apache.camel.Processor;
 import org.apache.camel.impl.ServiceSupport;
 import org.apache.camel.util.ServiceHelper;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 /**
  * A default base class for a {@link LoadBalancer} implementation
@@ -33,6 +35,7 @@ import org.apache.camel.util.ServiceHelp
 public abstract class LoadBalancerSupport extends ServiceSupport implements LoadBalancer, Navigate<Processor> {
 
     private final List<Processor> processors = new CopyOnWriteArrayList<Processor>();
+    protected final Log log = LogFactory.getLog(getClass());
 
     public void addProcessor(Processor processor) {
         processors.add(processor);

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/ContextTestSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/ContextTestSupport.java?rev=921281&r1=921280&r2=921281&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/ContextTestSupport.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/ContextTestSupport.java Wed Mar 10 10:14:27 2010
@@ -286,6 +286,13 @@ public abstract class ContextTestSupport
         MockEndpoint.assertIsSatisfied(context);
     }
 
+    /**
+     * Reset all Mock endpoints.
+     */
+    protected void resetMocks() {
+        MockEndpoint.resetMocks(context);
+    }
+
     protected void assertValidContext(CamelContext context) {
         assertNotNull("No context found!", context);
     }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java?rev=921281&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java Wed Mar 10 10:14:27 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.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class FailoverMaximumFailoverAttemptsTest extends ContextTestSupport {
+
+    public void testFailoverMaximumFailoverAttempts() throws Exception {
+        getMockEndpoint("mock:bad").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bad2").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bad3").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:good").expectedMessageCount(0);
+
+        try {
+            template.sendBody("direct:start", "Hello World");
+            fail("Should throw exception");
+        } catch (Exception e) {
+            assertEquals("Damn Again Again", e.getCause().getMessage());
+        }
+
+        assertMockEndpointsSatisfied();
+
+        // as its round robin based it remembers that last good endpoint
+        // and will invoke the next
+
+        resetMocks();
+
+        getMockEndpoint("mock:bad").expectedMessageCount(0);
+        getMockEndpoint("mock:bad2").expectedMessageCount(0);
+        getMockEndpoint("mock:bad3").expectedMessageCount(0);
+        getMockEndpoint("mock:good").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Bye World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .loadBalance().failover(2, false, true).
+                        to("direct:bad", "direct:bad2", "direct:bad3", "direct:good");
+
+                from("direct:bad")
+                    .to("mock:bad")
+                    .throwException(new IllegalArgumentException("Damn"));
+
+                from("direct:bad2")
+                    .to("mock:bad2")
+                    .throwException(new IllegalArgumentException("Damn Again"));
+
+                from("direct:bad3")
+                    .to("mock:bad3")
+                    .throwException(new IllegalArgumentException("Damn Again Again"));
+
+                from("direct:good")
+                    .to("mock:good");
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverMaximumFailoverAttemptsTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java?rev=921281&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java Wed Mar 10 10:14:27 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.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class FailoverRoundRobinTest extends ContextTestSupport {
+
+    public void testFailoverRoundRobin() throws Exception {
+        getMockEndpoint("mock:bad").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bad2").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:good").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:good2").expectedMessageCount(0);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+
+        // as its round robin based it remembers that last good endpoint
+        // and will invoke the next
+
+        resetMocks();
+
+        getMockEndpoint("mock:bad").expectedMessageCount(0);
+        getMockEndpoint("mock:bad2").expectedMessageCount(0);
+        getMockEndpoint("mock:good").expectedMessageCount(0);
+        getMockEndpoint("mock:good2").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Bye World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // START SNIPPET: e1
+                from("direct:start")
+                    // Use failover load balancer in stateful round robin mode
+                    // which mean it will failover immediately in case of an exception
+                    // as it does NOT inherit error handler. It will also keep retrying as
+                    // its configured to newer exhaust.
+                    .loadBalance().failover(-1, false, true).
+                        to("direct:bad", "direct:bad2", "direct:good", "direct:good2");
+                // END SNIPPET: e1
+
+                from("direct:bad")
+                    .to("mock:bad")
+                    .throwException(new IllegalArgumentException("Damn"));
+
+                from("direct:bad2")
+                    .to("mock:bad2")
+                    .throwException(new IllegalArgumentException("Damn Again"));
+
+                from("direct:good")
+                    .to("mock:good");
+
+                from("direct:good2")
+                    .to("mock:good2");
+            }
+        };
+    }
+
+}

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FailoverRoundRobinTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverMaximumFailoverAttemptsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverMaximumFailoverAttemptsTest.java?rev=921281&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverMaximumFailoverAttemptsTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverMaximumFailoverAttemptsTest.java Wed Mar 10 10:14:27 2010
@@ -0,0 +1,32 @@
+/**
+ * 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.
+ */
+package org.apache.camel.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.FailoverMaximumFailoverAttemptsTest;
+
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision: 630591 $
+ */
+public class SpringFailoverMaximumFailoverAttemptsTest extends FailoverMaximumFailoverAttemptsTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml");
+    }
+}
\ No newline at end of file

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java (from r921228, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringAggregatorTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringAggregatorTest.java&r1=921228&r2=921281&rev=921281&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringAggregatorTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java Wed Mar 10 10:14:27 2010
@@ -17,14 +17,16 @@
 package org.apache.camel.spring.processor;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.processor.AggregatorTest;
+import org.apache.camel.processor.FailoverRoundRobinTest;
+
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
 /**
  * @version $Revision$
  */
-public class SpringAggregatorTest extends AggregatorTest {
+public class SpringFailoverRoundRobinTest extends FailoverRoundRobinTest {
+
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/aggregator.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/FailoverRoundRobinTest.xml");
     }
 }
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringFailoverRoundRobinTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml?rev=921281&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml Wed Mar 10 10:14:27 2010
@@ -0,0 +1,79 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <bean id="damn" class="java.lang.IllegalArgumentException">
+        <constructor-arg index="0" value="Damn"/>
+    </bean>
+
+    <bean id="damnAgain" class="java.lang.IllegalArgumentException">
+        <constructor-arg index="0" value="Damn Again"/>
+    </bean>
+
+    <bean id="damnAgainAgain" class="java.lang.IllegalArgumentException">
+        <constructor-arg index="0" value="Damn Again Again"/>
+    </bean>
+
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+
+        <!-- START SNIPPET: e1 -->
+        <route>
+            <from uri="direct:start"/>
+            <loadBalance>
+                <!-- failover using stateful round robin,
+                     which will at most failover 2 times before exhausting -->
+                <failover maximumFailoverAttempts="2" roundRobin="true"/>
+                <to uri="direct:bad"/>
+                <to uri="direct:bad2"/>
+                <to uri="direct:bad3"/>
+                <to uri="direct:good"/>
+            </loadBalance>
+        </route>
+        <!-- END SNIPPET: e1 -->
+
+        <route>
+            <from uri="direct:bad"/>
+            <to uri="mock:bad"/>
+            <throwException ref="damn"/>
+        </route>
+
+        <route>
+            <from uri="direct:bad2"/>
+            <to uri="mock:bad2"/>
+            <throwException ref="damnAgain"/>
+        </route>
+
+        <route>
+            <from uri="direct:bad3"/>
+            <to uri="mock:bad3"/>
+            <throwException ref="damnAgainAgain"/>
+        </route>
+
+        <route>
+            <from uri="direct:good"/>
+            <to uri="mock:good"/>
+        </route>
+
+    </camelContext>
+
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverMaximumFailoverAttemptsTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml (from r921228, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafter.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafter.xml&r1=921228&r2=921281&rev=921281&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafter.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml Wed Mar 10 10:14:27 2010
@@ -22,16 +22,54 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
     ">
 
-    <!-- START SNIPPET: e1 -->
+    <bean id="damn" class="java.lang.IllegalArgumentException">
+        <constructor-arg index="0" value="Damn"/>
+    </bean>
+
+    <bean id="damnAgain" class="java.lang.IllegalArgumentException">
+        <constructor-arg index="0" value="Damn Again"/>
+    </bean>
+
     <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+
+        <!-- START SNIPPET: e1 -->
         <route>
             <from uri="direct:start"/>
-            <aop afterUri="mock:after">
-                <transform><constant>Bye World</constant></transform>
-                <to uri="mock:result"/>
-            </aop>
+            <loadBalance>
+                <!-- failover using stateful round robin,
+                     which will keep retrying forever those 4 endpoints until success.
+                     You can set the maximumFailoverAttempt to break out after X attempts -->
+                <failover roundRobin="true"/>
+                <to uri="direct:bad"/>
+                <to uri="direct:bad2"/>
+                <to uri="direct:good"/>
+                <to uri="direct:good2"/>
+            </loadBalance>
+        </route>
+        <!-- END SNIPPET: e1 -->
+
+        <route>
+            <from uri="direct:bad"/>
+            <to uri="mock:bad"/>
+            <throwException ref="damn"/>
+        </route>
+
+        <route>
+            <from uri="direct:bad2"/>
+            <to uri="mock:bad2"/>
+            <throwException ref="damnAgain"/>
+        </route>
+
+        <route>
+            <from uri="direct:good"/>
+            <to uri="mock:good"/>
         </route>
+
+        <route>
+            <from uri="direct:good2"/>
+            <to uri="mock:good2"/>
+        </route>
+
     </camelContext>
-    <!-- END SNIPPET: e1 -->
 
 </beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/FailoverRoundRobinTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml