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 2012/02/11 13:30:33 UTC

svn commit: r1243046 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/ main/java/org/apache/camel/model/ main/java/org/apache/camel/processor/ test/java/org/apache/camel/processor/

Author: davsclaus
Date: Sat Feb 11 12:30:32 2012
New Revision: 1243046

URL: http://svn.apache.org/viewvc?rev=1243046&view=rev
Log:
CAMEL-5000: Fixed error handler not called when recipient list has non existing endpoint.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointErrorHandlerTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointIgnoreInvalidEndpointsTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/RecipientList.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MulticastProcessor.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/RecipientList.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/RecipientList.java?rev=1243046&r1=1243045&r2=1243046&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/RecipientList.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/RecipientList.java Sat Feb 11 12:30:32 2012
@@ -53,5 +53,6 @@ public @interface RecipientList {
     String executorServiceRef() default "";
     long timeout() default 0;
     String onPrepareRef() default "";
+    @Deprecated
     boolean shareUnitOfWork() default false;
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java?rev=1243046&r1=1243045&r2=1243046&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java Sat Feb 11 12:30:32 2012
@@ -254,17 +254,7 @@ public abstract class ProcessorDefinitio
                 // only wrap the parent (not the children of the multicast)
                 wrapChannelInErrorHandler(channel, routeContext);
             } else {
-                log.trace("{} is part of multicast/recipientList which have special error handling so no error handler is applied", defn);
-            }
-        } else if (defn instanceof RecipientListDefinition) {
-            // do not use error handler for recipient list as it offers fine grained error handlers for its outputs
-            // however if share unit of work is enabled, we need to wrap an error handler on the recipient list parent
-            RecipientListDefinition<?> def = (RecipientListDefinition<?>) defn;
-            if (def.isShareUnitOfWork()) {
-                // note a recipient list cannot have children so no need for a child == null check
-                wrapChannelInErrorHandler(channel, routeContext);
-            } else {
-                log.trace("{} is part of multicast/recipientList which have special error handling so no error handler is applied", defn);
+                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

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java?rev=1243046&r1=1243045&r2=1243046&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java Sat Feb 11 12:30:32 2012
@@ -71,6 +71,7 @@ public class RecipientListDefinition<Typ
     @XmlTransient
     private Processor onPrepare;
     @XmlAttribute
+    @Deprecated
     private Boolean shareUnitOfWork;
 
     public RecipientListDefinition() {
@@ -308,6 +309,7 @@ public class RecipientListDefinition<Typ
      * @return the builder.
      * @see org.apache.camel.spi.SubUnitOfWork
      */
+    @Deprecated
     public RecipientListDefinition<Type> shareUnitOfWork() {
         setShareUnitOfWork(true);
         return this;
@@ -428,14 +430,17 @@ public class RecipientListDefinition<Typ
         this.onPrepare = onPrepare;
     }
 
+    @Deprecated
     public Boolean getShareUnitOfWork() {
         return shareUnitOfWork;
     }
 
+    @Deprecated
     public void setShareUnitOfWork(Boolean shareUnitOfWork) {
         this.shareUnitOfWork = shareUnitOfWork;
     }
 
+    @Deprecated
     public boolean isShareUnitOfWork() {
         return shareUnitOfWork != null && shareUnitOfWork;
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MulticastProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MulticastProcessor.java?rev=1243046&r1=1243045&r2=1243046&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MulticastProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MulticastProcessor.java Sat Feb 11 12:30:32 2012
@@ -736,14 +736,14 @@ public class MulticastProcessor extends 
     protected void doDone(Exchange original, Exchange subExchange, AsyncCallback callback, boolean doneSync, boolean exhaust) {
         // cleanup any per exchange aggregation strategy
         removeAggregationStrategyFromExchange(original);
-        if (original.getException() != null) {
+        if (original.getException() != null || subExchange != null && subExchange.getException() != null) {
             // multicast uses error handling on its output processors and they have tried to redeliver
             // so we shall signal back to the other error handlers that we are exhausted and they should not
             // also try to redeliver as we will then do that twice
             original.setProperty(Exchange.REDELIVERY_EXHAUSTED, exhaust);
         }
         if (subExchange != null) {
-            // and copy the current result to original so it will contain this exception
+            // and copy the current result to original so it will contain this result of this eip
             ExchangeHelper.copyResults(original, subExchange);
         }
         callback.done(doneSync);

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointErrorHandlerTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointErrorHandlerTest.java?rev=1243046&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointErrorHandlerTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointErrorHandlerTest.java Sat Feb 11 12:30:32 2012
@@ -0,0 +1,47 @@
+/**
+ * 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;
+
+public class RecipientListInvalidEndpointErrorHandlerTest extends ContextTestSupport {
+
+    public void testRecipientListInvalidEndpointErrorHandler() 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() {
+                onException(Exception.class)
+                        .handled(true)
+                        .to("mock:dead");
+
+                from("direct:start")
+                    .recipientList(constant("fail:endpoint"))
+                    .to("mock:result");
+            }
+        };
+    }
+
+}

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointIgnoreInvalidEndpointsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointIgnoreInvalidEndpointsTest.java?rev=1243046&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointIgnoreInvalidEndpointsTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/RecipientListInvalidEndpointIgnoreInvalidEndpointsTest.java Sat Feb 11 12:30:32 2012
@@ -0,0 +1,47 @@
+/**
+ * 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;
+
+public class RecipientListInvalidEndpointIgnoreInvalidEndpointsTest extends ContextTestSupport {
+
+    public void testRecipientListInvalidEndpointIgnoreInvalidEndpoints() throws Exception {
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+        getMockEndpoint("mock:dead").expectedMessageCount(0);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+    
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                onException(Exception.class)
+                        .handled(true)
+                        .to("mock:dead");
+
+                from("direct:start")
+                    .recipientList(constant("fail:endpoint")).ignoreInvalidEndpoints()
+                    .to("mock:result");
+            }
+        };
+    }
+
+}