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 2016/11/11 13:32:15 UTC

camel git commit: CAMEL-10473: Failover Loadbalancer - Error handler should kick in after exhausted when inheritErrorHandler is false

Repository: camel
Updated Branches:
  refs/heads/camel-2.17.x 24176c52d -> 95a7ed63f


CAMEL-10473: Failover Loadbalancer - Error handler should kick in after exhausted when inheritErrorHandler is false


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/95a7ed63
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/95a7ed63
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/95a7ed63

Branch: refs/heads/camel-2.17.x
Commit: 95a7ed63fef83c5f49312d20f0eb013018807e8c
Parents: 24176c5
Author: Claus Ibsen <da...@apache.org>
Authored: Fri Nov 11 14:30:47 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Fri Nov 11 14:31:49 2016 +0100

----------------------------------------------------------------------
 .../camel/model/LoadBalanceDefinition.java      | 12 ++++-
 .../apache/camel/model/ProcessorDefinition.java | 17 +++---
 ...LoadBalanceNotInheritedErrorHandlerTest.java | 55 ++++++++++++++++++++
 3 files changed, 76 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/95a7ed63/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java b/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
index bc9370a..ece48c4 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
@@ -108,7 +108,7 @@ public class LoadBalanceDefinition extends ProcessorDefinition<LoadBalanceDefini
             loadBalancer = loadBalancerType.createLoadBalancer(routeContext);
             loadBalancerType.setLoadBalancer(loadBalancer);
 
-            // some load balancers can only support a fixed number of outputs
+            // some load balancer can only support a fixed number of outputs
             int max = loadBalancerType.getMaximumNumberOfOutputs();
             int size = getOutputs().size();
             if (size > max) {
@@ -127,7 +127,15 @@ public class LoadBalanceDefinition extends ProcessorDefinition<LoadBalanceDefini
                 loadBalancer.addProcessor(processor);
             }
         }
-        return loadBalancer;
+
+        Boolean inherit = inheritErrorHandler;
+        if (loadBalancerType instanceof FailoverLoadBalancerDefinition) {
+            // special for failover load balancer where you can configure it to not inherit error handler for its children
+            // but the load balancer itself should inherit so Camels error handler can react afterwards
+            inherit = true;
+        }
+        Processor target = wrapChannel(routeContext, loadBalancer, this, inherit);
+        return target;
     }
     
     // Fluent API

http://git-wip-us.apache.org/repos/asf/camel/blob/95a7ed63/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index 1ba4bb5..cf97690 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -258,6 +258,10 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
     }
 
     protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child) throws Exception {
+        return wrapChannel(routeContext, processor, child, isInheritErrorHandler());
+    }
+
+    protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child, Boolean inheritErrorHandler) throws Exception {
         // put a channel in between this and each output to control the route flow logic
         ModelChannel channel = createChannel(routeContext);
         channel.setNextProcessor(processor);
@@ -294,13 +298,13 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
             boolean isShareUnitOfWork = def.getShareUnitOfWork() != null && def.getShareUnitOfWork();
             if (isShareUnitOfWork && child == null) {
                 // only wrap the parent (not the children of the multicast)
-                wrapChannelInErrorHandler(channel, routeContext);
+                wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
             } else {
                 log.trace("{} is part of multicast which have special error handling so no error handler is applied", defn);
             }
         } else {
             // use error handler by default or if configured to do so
-            wrapChannelInErrorHandler(channel, routeContext);
+            wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
         }
 
         // do post init at the end
@@ -313,12 +317,13 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
     /**
      * Wraps the given channel in error handler (if error handler is inherited)
      *
-     * @param channel       the channel
-     * @param routeContext  the route context
+     * @param channel             the channel
+     * @param routeContext        the route context
+     * @param inheritErrorHandler whether to inherit error handler
      * @throws Exception can be thrown if failed to create error handler builder
      */
-    private void wrapChannelInErrorHandler(Channel channel, RouteContext routeContext) throws Exception {
-        if (isInheritErrorHandler() == null || isInheritErrorHandler()) {
+    private void wrapChannelInErrorHandler(Channel channel, RouteContext routeContext, Boolean inheritErrorHandler) throws Exception {
+        if (inheritErrorHandler == null || inheritErrorHandler) {
             log.trace("{} is configured to inheritErrorHandler", this);
             Processor output = channel.getOutput();
             Processor errorHandler = wrapInErrorHandler(routeContext, output);

http://git-wip-us.apache.org/repos/asf/camel/blob/95a7ed63/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalanceNotInheritedErrorHandlerTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalanceNotInheritedErrorHandlerTest.java b/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalanceNotInheritedErrorHandlerTest.java
new file mode 100644
index 0000000..6a8ed0d
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/processor/FailOverLoadBalanceNotInheritedErrorHandlerTest.java
@@ -0,0 +1,55 @@
+/**
+ * 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 java.io.IOException;
+import java.net.ConnectException;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+public class FailOverLoadBalanceNotInheritedErrorHandlerTest extends ContextTestSupport {
+
+    public void testFailOverLoadBalanceNotInheritedErrorHandler() throws Exception {
+        getMockEndpoint("mock:result").expectedMessageCount(0);
+        getMockEndpoint("mock:dead").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // after failover is done, then we should be routed to dead
+                errorHandler(deadLetterChannel("mock:dead"));
+
+                from("direct:start")
+                    .loadBalance()
+                        .failover(3, false, true)
+                            .throwException(new IllegalArgumentException())
+                            .throwException(new IOException())
+                            .throwException(new ConnectException())
+                        .end()
+                    .end()
+                    .to("mock:result");
+            }
+        };
+    }
+
+}