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 2013/04/25 09:53:11 UTC

svn commit: r1475657 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/processor/ camel-core/src/test/java/org/apache/camel/processor/ camel-core/src/test/java/org/apache/camel/processor/onexception/ components/camel-ftp/src/test/java/org/ap...

Author: davsclaus
Date: Thu Apr 25 07:53:10 2013
New Revision: 1475657

URL: http://svn.apache.org/r1475657
Log:
CAMEL-6311: DLC should always handle exception. When using a custom onException that may cause DLC to not handle because it didn't do that. So this is now fixed, to ensure always handle when using DLC, as thats the point of this eip.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelAlwaysHandledTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RedeliveryErrorHandler.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/BeanWithExceptionTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SplitterTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexRouteTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpDoNotDeleteFileIfProcessFailsTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpThirdPoolOkTest.java
    camel/trunk/components/camel-mail/src/test/java/org/apache/camel/component/mail/MailDoNotDeleteIfProcessFailsTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java Thu Apr 25 07:53:10 2013
@@ -86,4 +86,8 @@ public class DeadLetterChannel extends R
         return true;
     }
 
+    @Override
+    public boolean isDeadLetterChannel() {
+        return true;
+    }
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java Thu Apr 25 07:53:10 2013
@@ -107,6 +107,13 @@ public abstract class ErrorHandlerSuppor
     public abstract boolean supportTransacted();
 
     /**
+     * Whether this error handler handles exhausted errors by moving the exchange to a dead letter channel.
+     */
+    public boolean isDeadLetterChannel() {
+        return false;
+    }
+
+    /**
      * Gets the output
      */
     public abstract Processor getOutput();

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RedeliveryErrorHandler.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RedeliveryErrorHandler.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RedeliveryErrorHandler.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RedeliveryErrorHandler.java Thu Apr 25 07:53:10 2013
@@ -326,7 +326,8 @@ public abstract class RedeliveryErrorHan
                 }
                 // we should always invoke the deliverToFailureProcessor as it prepares, logs and does a fair
                 // bit of work for exhausted exchanges (its only the target processor which may be null if handled by a savepoint)
-                boolean sync = deliverToFailureProcessor(target, exchange, data, callback);
+                boolean isDeadLetterChannel = isDeadLetterChannel() && target == data.deadLetterProcessor;
+                boolean sync = deliverToFailureProcessor(target, isDeadLetterChannel, exchange, data, callback);
                 // we are breaking out
                 return sync;
             }
@@ -496,7 +497,8 @@ public abstract class RedeliveryErrorHan
             }
             // we should always invoke the deliverToFailureProcessor as it prepares, logs and does a fair
             // bit of work for exhausted exchanges (its only the target processor which may be null if handled by a savepoint)
-            deliverToFailureProcessor(target, exchange, data, callback);
+            boolean isDeadLetterChannel = isDeadLetterChannel() && target == data.deadLetterProcessor;
+            deliverToFailureProcessor(target, isDeadLetterChannel, exchange, data, callback);
             // we are breaking out
             return;
         }
@@ -761,7 +763,7 @@ public abstract class RedeliveryErrorHan
     /**
      * All redelivery attempts failed so move the exchange to the dead letter queue
      */
-    protected boolean deliverToFailureProcessor(final Processor processor, final Exchange exchange,
+    protected boolean deliverToFailureProcessor(final Processor processor, final boolean isDeadLetterChannel, final Exchange exchange,
                                                 final RedeliveryData data, final AsyncCallback callback) {
         boolean sync = true;
 
@@ -771,7 +773,8 @@ public abstract class RedeliveryErrorHan
         // clear exception as we let the failure processor handle it
         exchange.setException(null);
 
-        final boolean shouldHandle = shouldHandled(exchange, data);
+        // always handle if dead letter channel
+        final boolean shouldHandle = isDeadLetterChannel || shouldHandled(exchange, data);
         final boolean shouldContinue = shouldContinue(exchange, data);
         // regard both handled or continued as being handled
         boolean handled = false;

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/BeanWithExceptionTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/BeanWithExceptionTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/BeanWithExceptionTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/BeanWithExceptionTest.java Thu Apr 25 07:53:10 2013
@@ -89,8 +89,6 @@ public class BeanWithExceptionTest exten
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                errorHandler(deadLetterChannel("mock:error"));
-
                 onException(ValidationException.class).to("mock:invalid");
 
                 from("direct:start").beanRef("myBean").to("mock:valid");

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelAlwaysHandledTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelAlwaysHandledTest.java?rev=1475657&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelAlwaysHandledTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelAlwaysHandledTest.java Thu Apr 25 07:53:10 2013
@@ -0,0 +1,85 @@
+/**
+ * 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.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.seda.SedaConsumer;
+import org.apache.camel.spi.ExceptionHandler;
+
+public class DeadLetterChannelAlwaysHandledTest extends ContextTestSupport {
+
+    private static final AtomicBoolean called = new AtomicBoolean();
+
+    public void testDeadLetterChannelAlwaysHandled() throws Exception {
+        // need to set exception handler manually to work around an issue configuring from uri
+        SedaConsumer seda = (SedaConsumer) context.getRoute("foo").getConsumer();
+        seda.setExceptionHandler(new MyExceptionHandler());
+
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        getMockEndpoint("mock:bar").expectedMessageCount(1);
+        getMockEndpoint("mock:dead").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(0);
+
+        template.sendBody("seda:foo", "Hello World");
+
+        assertMockEndpointsSatisfied();
+
+        assertFalse("Should not have called", called.get());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                errorHandler(deadLetterChannel("mock:dead"));
+
+                from("seda:foo?synchronous=true").routeId("foo")
+                    .to("mock:foo")
+                    .to("direct:bar")
+                    .to("mock:result");
+
+                from("direct:bar").routeId("bar")
+                    .onException(IllegalArgumentException.class).maximumRedeliveries(3).redeliveryDelay(0).end()
+                    .to("mock:bar")
+                    .throwException(new IllegalArgumentException("Forced"));
+            }
+        };
+    }
+
+    private final class MyExceptionHandler implements ExceptionHandler {
+
+        @Override
+        public void handleException(Throwable exception) {
+            called.set(true);
+        }
+
+        @Override
+        public void handleException(String message, Throwable exception) {
+            called.set(true);
+        }
+
+        @Override
+        public void handleException(String message, Exchange exchange, Throwable exception) {
+            called.set(true);
+        }
+    }
+}

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SplitterTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SplitterTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SplitterTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SplitterTest.java Thu Apr 25 07:53:10 2013
@@ -246,9 +246,7 @@ public class SplitterTest extends Contex
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                errorHandler(deadLetterChannel("mock:failed").maximumRedeliveries(0));
-                // we don't want the DLC to handle the Exception
-                onException(CamelException.class).handled(false);
+                onException(CamelException.class).to("mock:failed");
 
                 from("direct:seqential").split(body().tokenize(","), new UseLatestAggregationStrategy()).to("mock:result");
                 from("direct:parallel").split(body().tokenize(","), new MyAggregationStrategy()).parallelProcessing().to("mock:result");

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexRouteTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexRouteTest.java Thu Apr 25 07:53:10 2013
@@ -47,12 +47,8 @@ public class OnExceptionComplexRouteTest
         getMockEndpoint("mock:error").expectedMessageCount(1);
         getMockEndpoint("mock:result").expectedMessageCount(0);
 
-        try {
-            template.sendBody("direct:start", "<order><type>myType</type><user>Func</user></order>");
-            fail("Should have thrown a MyFunctionalException");
-        } catch (RuntimeCamelException e) {
-            assertIsInstanceOf(MyFunctionalException.class, e.getCause());
-        }
+        // we use DLC so all exceptions gets handled
+        template.sendBody("direct:start", "<order><type>myType</type><user>Func</user></order>");
 
         assertMockEndpointsSatisfied();
     }

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpDoNotDeleteFileIfProcessFailsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpDoNotDeleteFileIfProcessFailsTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpDoNotDeleteFileIfProcessFailsTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpDoNotDeleteFileIfProcessFailsTest.java Thu Apr 25 07:53:10 2013
@@ -80,8 +80,9 @@ public class FromFtpDoNotDeleteFileIfPro
         return new RouteBuilder() {
             public void configure() throws Exception {
                 // use no delay for fast unit testing
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(2).redeliveryDelay(0));
-                onException(IllegalArgumentException.class).handled(false);     // DLC should not handle
+                onException(IllegalArgumentException.class)
+                    .maximumRedeliveries(2).redeliveryDelay(0)
+                    .to("mock:error");
 
                 from(getFtpUrl()).process(new Processor() {
                     public void process(Exchange exchange) throws Exception {

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpThirdPoolOkTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpThirdPoolOkTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpThirdPoolOkTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpThirdPoolOkTest.java Thu Apr 25 07:53:10 2013
@@ -39,7 +39,7 @@ public class FromFtpThirdPoolOkTest exte
     @Override
     @Before
     public void setUp() throws Exception {
-        deleteDirectory("target/thridpool");
+        deleteDirectory("target/thirdpool");
         super.setUp();
     }
 
@@ -67,8 +67,10 @@ public class FromFtpThirdPoolOkTest exte
         return new RouteBuilder() {
             public void configure() throws Exception {
                 // no redeliveries as we want the ftp consumer to try again
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(0).logStackTrace(false));
-                onException(IllegalArgumentException.class).handled(false);     // DLC should not handle
+                // use no delay for fast unit testing
+                onException(IllegalArgumentException.class)
+                        .logStackTrace(false)
+                        .to("mock:error");
 
                 from(getFtpUrl()).process(new Processor() {
                     public void process(Exchange exchange) throws Exception {

Modified: camel/trunk/components/camel-mail/src/test/java/org/apache/camel/component/mail/MailDoNotDeleteIfProcessFailsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-mail/src/test/java/org/apache/camel/component/mail/MailDoNotDeleteIfProcessFailsTest.java?rev=1475657&r1=1475656&r2=1475657&view=diff
==============================================================================
--- camel/trunk/components/camel-mail/src/test/java/org/apache/camel/component/mail/MailDoNotDeleteIfProcessFailsTest.java (original)
+++ camel/trunk/components/camel-mail/src/test/java/org/apache/camel/component/mail/MailDoNotDeleteIfProcessFailsTest.java Thu Apr 25 07:53:10 2013
@@ -80,8 +80,7 @@ public class MailDoNotDeleteIfProcessFai
         return new RouteBuilder() {
             public void configure() throws Exception {
                 // no redelivery for unit test as we want it to be polled next time
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(0).logStackTrace(false));
-                onException(IllegalArgumentException.class).handled(false); // DLC should not handle
+                onException(IllegalArgumentException.class).to("mock:error");
 
                 from("imap://localhost?username=claus&password=secret&unseen=true&delay=250")
                         .process(new Processor() {