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 2009/04/03 15:05:12 UTC

svn commit: r761660 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/builder/ main/java/org/apache/camel/processor/ test/java/org/apache/camel/processor/ test/java/org/apache/camel/processor/onexception/

Author: davsclaus
Date: Fri Apr  3 13:05:11 2009
New Revision: 761660

URL: http://svn.apache.org/viewvc?rev=761660&view=rev
Log:
CAMEL-1511: Added onException support to DefaultErrorHandler.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerOnExceptionTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceWithDefaultErrorHandlerTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformWithDefaultErrorHandlerTest.java
      - copied, changed from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DeadLetterChannelBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DefaultErrorHandlerBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultErrorHandler.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DeadLetterChannelBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DeadLetterChannelBuilder.java?rev=761660&r1=761659&r2=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DeadLetterChannelBuilder.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DeadLetterChannelBuilder.java Fri Apr  3 13:05:11 2009
@@ -76,6 +76,8 @@
 
     public Processor createErrorHandler(RouteContext routeContext, Processor processor) throws Exception {
         DeadLetterChannel answer = new DeadLetterChannel(processor, getFailureProcessor(), deadLetterUri, onRedelivery, getRedeliveryPolicy(), getLogger(), getExceptionPolicyStrategy());
+        // must enable stream cache as DeadLetterChannel can do redeliveries and
+        // thus it needs to be able to read the stream again
         StreamCaching.enable(routeContext);
         configure(answer);
         return answer;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DefaultErrorHandlerBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DefaultErrorHandlerBuilder.java?rev=761660&r1=761659&r2=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DefaultErrorHandlerBuilder.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DefaultErrorHandlerBuilder.java Fri Apr  3 13:05:11 2009
@@ -18,6 +18,8 @@
 
 import org.apache.camel.Processor;
 import org.apache.camel.processor.DefaultErrorHandler;
+import org.apache.camel.processor.ErrorHandlerSupport;
+import org.apache.camel.processor.exceptionpolicy.ExceptionPolicyStrategy;
 import org.apache.camel.spi.RouteContext;
 
 /**
@@ -27,10 +29,32 @@
  */
 public class DefaultErrorHandlerBuilder extends ErrorHandlerBuilderSupport {
 
-    // TODO: in the future support onException
+    private ExceptionPolicyStrategy exceptionPolicyStrategy = ErrorHandlerSupport.createDefaultExceptionPolicyStrategy();
 
     public Processor createErrorHandler(RouteContext routeContext, Processor processor) {
-        return new DefaultErrorHandler(processor);
+        DefaultErrorHandler answer = new DefaultErrorHandler(processor, exceptionPolicyStrategy);
+        configure(answer);
+        return answer;
+    }
+
+    /**
+     * Sets the exception policy to use
+     */
+    public DefaultErrorHandlerBuilder exceptionPolicyStrategy(ExceptionPolicyStrategy exceptionPolicyStrategy) {
+        setExceptionPolicyStrategy(exceptionPolicyStrategy);
+        return this;
+    }
+
+    /**
+     * Sets the exception policy strategy to use for resolving the {@link org.apache.camel.model.OnExceptionDefinition}
+     * to use for a given thrown exception
+     */
+    public ExceptionPolicyStrategy getExceptionPolicyStrategy() {
+        return exceptionPolicyStrategy;
+    }
+
+    public void setExceptionPolicyStrategy(ExceptionPolicyStrategy exceptionPolicyStrategy) {
+        this.exceptionPolicyStrategy = exceptionPolicyStrategy;
     }
 
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultErrorHandler.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultErrorHandler.java?rev=761660&r1=761659&r2=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultErrorHandler.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultErrorHandler.java Fri Apr  3 13:05:11 2009
@@ -19,9 +19,16 @@
 import org.apache.camel.AsyncCallback;
 import org.apache.camel.AsyncProcessor;
 import org.apache.camel.Exchange;
+import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
 import org.apache.camel.impl.converter.AsyncProcessorTypeConverter;
+import org.apache.camel.model.OnExceptionDefinition;
+import org.apache.camel.processor.exceptionpolicy.ExceptionPolicyStrategy;
+import org.apache.camel.util.ExchangeHelper;
+import org.apache.camel.util.MessageHelper;
 import org.apache.camel.util.ServiceHelper;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 /**
  * Default error handler
@@ -30,12 +37,14 @@
  */
 public class DefaultErrorHandler extends ErrorHandlerSupport implements AsyncProcessor {
 
+    private static final transient Log LOG = LogFactory.getLog(DefaultErrorHandler.class);
     private Processor output;
     private AsyncProcessor outputAsync;
 
-    public DefaultErrorHandler(Processor output) {
+    public DefaultErrorHandler(Processor output, ExceptionPolicyStrategy exceptionPolicyStrategy) {
         this.output = output;
         this.outputAsync = AsyncProcessorTypeConverter.convert(output);
+        setExceptionPolicy(exceptionPolicyStrategy);
     }
 
     @Override
@@ -47,14 +56,76 @@
         output.process(exchange);
     }
 
-    public boolean process(Exchange exchange, final AsyncCallback callback) {
+    public boolean process(final Exchange exchange, final AsyncCallback callback) {
         return outputAsync.process(exchange, new AsyncCallback() {
             public void done(boolean sync) {
+                if (exchange.getException() != null && !ExchangeHelper.isFailureHandled(exchange)) {
+                    handleException(exchange);
+                }
+
                 callback.done(sync);
             }
         });
     }
 
+    private void handleException(Exchange exchange) {
+        Exception e = exchange.getException();
+
+        // store the original caused exception in a property, so we can restore it later
+        exchange.setProperty(Exchange.EXCEPTION_CAUGHT, e);
+
+        // find the error handler to use (if any)
+        OnExceptionDefinition exceptionPolicy = getExceptionPolicy(exchange, e);
+        if (exceptionPolicy != null) {
+            Predicate handledPredicate = exceptionPolicy.getHandledPolicy();
+
+            Processor processor = exceptionPolicy.getErrorHandler();
+            if (processor != null) {
+                prepareExchangeBeforeOnException(exchange);
+                deliverToFaultProcessor(exchange, processor);
+                prepareExchangeAfterOnException(exchange, handledPredicate);
+            }
+        }
+    }
+
+    private void prepareExchangeBeforeOnException(Exchange exchange) {
+        // okay lower the exception as we are handling it by onException
+        if (exchange.getException() != null) {
+            exchange.setException(null);
+        }
+
+        // clear rollback flags
+        exchange.setProperty(Exchange.ROLLBACK_ONLY, null);
+
+        // reset cached streams so they can be read again
+        MessageHelper.resetStreamCache(exchange.getIn());
+    }
+
+    private boolean deliverToFaultProcessor(final Exchange exchange, final Processor failureProcessor) {
+        AsyncProcessor afp = AsyncProcessorTypeConverter.convert(failureProcessor);
+        boolean sync = afp.process(exchange, new AsyncCallback() {
+            public void done(boolean sync) {
+            }
+        });
+
+        return sync;
+    }
+
+    private void prepareExchangeAfterOnException(Exchange exchange, Predicate handledPredicate) {
+        if (handledPredicate == null || !handledPredicate.matches(exchange)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("This exchange is not handled so its marked as failed: " + exchange);
+            }
+            // exception not handled, put exception back in the exchange
+            exchange.setException(exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class));
+        } else {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("This exchange is handled so its marked as not failed: " + exchange);
+            }
+            exchange.setProperty(Exchange.EXCEPTION_HANDLED, Boolean.TRUE);
+        }
+    }
+
     /**
      * Returns the output processor
      */

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java?rev=761660&r1=761659&r2=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java Fri Apr  3 13:05:11 2009
@@ -22,18 +22,16 @@
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 
-
 public class DeadLetterChannelExceptionCausePropagatedTest extends ContextTestSupport {
-    private static final RuntimeException RUNTIME_EXCEPTION = new RuntimeException("Expected exception.");
-    private String body = "<hello>world!</hello>";
+    protected static final RuntimeException RUNTIME_EXCEPTION = new RuntimeException("Expected exception.");
+    protected String body = "<hello>world!</hello>";
 
     public void testFirstFewAttemptsFail() throws Exception {
         MockEndpoint failedEndpoint = getMockEndpoint("mock:failed");
         MockEndpoint successEndpoint = getMockEndpoint("mock:success");
 
         failedEndpoint.expectedBodiesReceived(body);
-        failedEndpoint.message(0).header(Exchange.EXCEPTION_CAUGHT)
-            .isEqualTo(RUNTIME_EXCEPTION);
+        failedEndpoint.message(0).header(Exchange.EXCEPTION_CAUGHT).isEqualTo(RUNTIME_EXCEPTION);
         failedEndpoint.expectedMessageCount(1);
 
         successEndpoint.expectedMessageCount(0);
@@ -51,7 +49,8 @@
 
                 onException(RuntimeException.class).handled(true).to("mock:failed");
 
-                from("direct:start").process(ProcessorBuilder.throwException(RUNTIME_EXCEPTION))
+                from("direct:start")
+                    .process(ProcessorBuilder.throwException(RUNTIME_EXCEPTION))
                     .to("mock:success");
             }
         };

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest.java Fri Apr  3 13:05:11 2009
@@ -16,45 +16,21 @@
  */
 package org.apache.camel.processor;
 
-import org.apache.camel.ContextTestSupport;
-import org.apache.camel.Exchange;
 import org.apache.camel.builder.ProcessorBuilder;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.mock.MockEndpoint;
 
-
-public class DeadLetterChannelExceptionCausePropagatedTest extends ContextTestSupport {
-    private static final RuntimeException RUNTIME_EXCEPTION = new RuntimeException("Expected exception.");
-    private String body = "<hello>world!</hello>";
-
-    public void testFirstFewAttemptsFail() throws Exception {
-        MockEndpoint failedEndpoint = getMockEndpoint("mock:failed");
-        MockEndpoint successEndpoint = getMockEndpoint("mock:success");
-
-        failedEndpoint.expectedBodiesReceived(body);
-        failedEndpoint.message(0).header(Exchange.EXCEPTION_CAUGHT)
-            .isEqualTo(RUNTIME_EXCEPTION);
-        failedEndpoint.expectedMessageCount(1);
-
-        successEndpoint.expectedMessageCount(0);
-
-        sendBody("direct:start", body);
-
-        assertMockEndpointsSatisfied();
-        assertNull(failedEndpoint.getExchanges().get(0).getException());
-    }
+public class DeadLetterChannelExceptionCausePropagatedWithDefaultErrorHandlerTest extends DeadLetterChannelExceptionCausePropagatedTest {
 
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                errorHandler(deadLetterChannel("mock:error").delay(0).maximumRedeliveries(3));
-
                 onException(RuntimeException.class).handled(true).to("mock:failed");
 
-                from("direct:start").process(ProcessorBuilder.throwException(RUNTIME_EXCEPTION))
+                from("direct:start")
+                    .process(ProcessorBuilder.throwException(RUNTIME_EXCEPTION))
                     .to("mock:success");
             }
         };
     }
 
-}
+}
\ No newline at end of file

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest.java Fri Apr  3 13:05:11 2009
@@ -16,47 +16,12 @@
  */
 package org.apache.camel.processor;
 
-import java.util.Map;
-
-import org.apache.camel.Body;
-import org.apache.camel.ContextTestSupport;
-import org.apache.camel.Headers;
-import org.apache.camel.OutHeaders;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.mock.MockEndpoint;
 
 /**
  * Unit test to verify that handled policy is working as expected for wiki documentation.
  */
-public class DeadLetterChannelHandledExampleTest extends ContextTestSupport {
-
-    public void testOrderOK() throws Exception {
-        MockEndpoint result = getMockEndpoint("mock:result");
-        result.expectedBodiesReceived("Order OK");
-        result.expectedHeaderReceived("orderid", "123");
-
-        MockEndpoint error = getMockEndpoint("mock:error");
-        error.expectedMessageCount(0);
-
-        Object out = template.requestBodyAndHeader("direct:start", "Order: MacBook Pro", "customerid", "444");
-        assertEquals("Order OK", out);
-
-        assertMockEndpointsSatisfied();
-    }
-
-    public void testOrderERROR() throws Exception {
-        MockEndpoint error = getMockEndpoint("mock:error");
-        error.expectedBodiesReceived("Order ERROR");
-        error.expectedHeaderReceived("orderid", "failed");
-
-        MockEndpoint result = getMockEndpoint("mock:result");
-        result.expectedMessageCount(0);
-
-        Object out = template.requestBodyAndHeader("direct:start", "Order: kaboom", "customerid", "555");
-        assertEquals("Order ERROR", out);
-
-        assertMockEndpointsSatisfied();
-    }
+public class DeadLetterChannelHandledExampleWithDefaultErrorHandlerTest extends DeadLetterChannelHandledExampleTest {
 
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
@@ -73,10 +38,6 @@
                     // and since this is an unit test we use mocks for testing
                     .to("mock:error");
 
-                // this is just the generic error handler where we set the destination
-                // and the number of redeliveries we want to try
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(1));
-
                 // this is our route where we handle orders
                 from("direct:start")
                     // this bean is our order service
@@ -88,58 +49,4 @@
         };
     }
 
-    // START SNIPPET: e2
-    /**
-     * Order service as a plain POJO class
-     */
-    public static class OrderService {
-
-        /**
-         * This method handle our order input and return the order
-         *
-         * @param in      the in headers
-         * @param payload the in payload
-         * @param out     the out headers
-         * @return the out payload
-         * @throws OrderFailedException is thrown if the order cannot be processed
-         */
-        public Object handleOrder(@Headers Map in, @Body String payload, @OutHeaders Map out)
-            throws OrderFailedException {
-            out.put("customerid", in.get("customerid"));
-            if ("Order: kaboom".equals(payload)) {
-                throw new OrderFailedException("Cannot order: kaboom");
-            } else {
-                out.put("orderid", "123");
-                return "Order OK";
-            }
-        }
-
-        /**
-         * This method creates the response to the caller if the order could not be processed
-         * @param in      the in headers
-         * @param payload the in payload
-         * @param out     the out headers
-         * @return the out payload
-         */
-        public Object orderFailed(@Headers Map in, @Body String payload, @OutHeaders Map out) {
-            out.put("customerid", in.get("customerid"));
-            out.put("orderid", "failed");
-            return "Order ERROR";
-        }
-    }
-    // END SNIPPET: e2
-
-    // START SNIPPET: e3
-    /**
-     * Exception thrown if the order cannot be processed
-     */
-    public static class OrderFailedException extends Exception {
-
-        public OrderFailedException(String message) {
-            super(message);
-        }
-        
-    }
-    // END SNIPPET: e3
-
 }
\ No newline at end of file

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerOnExceptionTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerOnExceptionTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerOnExceptionTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DefaultErrorHandlerOnExceptionTest.java Fri Apr  3 13:05:11 2009
@@ -19,32 +19,15 @@
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
-import org.apache.camel.Route;
-import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.impl.EventDrivenConsumerRoute;
 
 /**
  * Default error handler test
  *
  * @version $Revision$
  */
-public class DefaultErrorHandlerTest extends ContextTestSupport {
-
-    public void testRoute() {
-        Route route = context.getRoutes().get(0);
-        EventDrivenConsumerRoute consumerRoute = assertIsInstanceOf(EventDrivenConsumerRoute.class, route);
-
-        Processor processor = unwrap(consumerRoute.getProcessor());
-        Pipeline pipeline = assertIsInstanceOf(Pipeline.class, processor);
-
-        // there should be a default error handler in front of each processor in this pipeline
-        for (Processor child : pipeline.getProcessors()) {
-            DefaultErrorHandler errorHandler = assertIsInstanceOf(DefaultErrorHandler.class, child);
-            assertNotNull(errorHandler);
-        }
-    }
+public class DefaultErrorHandlerOnExceptionTest extends ContextTestSupport {
 
     public void testOk() throws Exception {
         MockEndpoint mock = getMockEndpoint("mock:result");
@@ -56,16 +39,11 @@
     }
 
     public void testWithError() throws Exception {
-        MockEndpoint mock = getMockEndpoint("mock:result");
-        mock.expectedMessageCount(0);
+        MockEndpoint mock = getMockEndpoint("mock:boom");
+        mock.expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Kabom");
 
-        try {
-            template.sendBody("direct:start", "Kabom");
-            fail("Should have thrown a RuntimeCamelException");
-        } catch (RuntimeCamelException e) {
-            // expected
-        }
-        
         assertMockEndpointsSatisfied();
     }
 
@@ -74,8 +52,7 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                // if no error handler is configured it should
-                // use the default error handler
+                onException(IllegalArgumentException.class).handled(true).to("mock:boom");
 
                 from("direct:start").process(new Processor() {
                     public void process(Exchange exchange) throws Exception {
@@ -89,4 +66,4 @@
             }
         };
     }
-}
+}
\ No newline at end of file

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest.java Fri Apr  3 13:05:11 2009
@@ -16,26 +16,23 @@
  */
 package org.apache.camel.processor.onexception;
 
-import java.io.IOException;
+import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.builder.RouteBuilder;
 
-public class OnExceptionComplexWithNestedErrorHandlerRouteTest extends OnExceptionComplexRouteTest {
+public class OnExceptionComplexWithNestedErrorHandlerRouteWithDefaultErrorHandlerTest extends OnExceptionComplexWithNestedErrorHandlerRouteTest {
 
-    public void testNoError3() throws Exception {
+    public void testFunctionalError() throws Exception {
+        // override as we dont support redelivery with DefaultErrorHandler
+        // then mock error should not receive any messages
         getMockEndpoint("mock:error").expectedMessageCount(0);
-        getMockEndpoint("mock:result").expectedMessageCount(1);
-
-        template.sendBody("direct:start3", "<order><type>myType</type><user>James</user></order>");
-
-        assertMockEndpointsSatisfied();
-    }
-
-    public void testFunctionalError3() throws Exception {
-        getMockEndpoint("mock:error").expectedMessageCount(0);
-        getMockEndpoint("mock:error3").expectedMessageCount(1);
         getMockEndpoint("mock:result").expectedMessageCount(0);
 
-        template.sendBody("direct:start3", "<order><type>myType</type><user>Func</user></order>");
+        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());
+        }
 
         assertMockEndpointsSatisfied();
     }
@@ -45,16 +42,13 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                // global error handler
-                errorHandler(deadLetterChannel("mock:error"));
-
                 // shared for both routes
-                onException(MyTechnicalException.class).handled(true).maximumRedeliveries(2).to("mock:tech.error");
+                onException(MyTechnicalException.class).handled(true).to("mock:tech.error");
 
                 from("direct:start")
                     // route specific on exception for MyFunctionalException
                     // we MUST use .end() to indicate that this sub block is ended
-                    .onException(MyFunctionalException.class).maximumRedeliveries(0).end()
+                    .onException(MyFunctionalException.class).end()
                     .to("bean:myServiceBean")
                     .to("mock:result");
 
@@ -62,7 +56,7 @@
                     // route specific on exception for MyFunctionalException that is different than the previous route
                     // here we marked it as handled and send it to a different destination mock:handled
                     // we MUST use .end() to indicate that this sub block is ended
-                    .onException(MyFunctionalException.class).handled(true).maximumRedeliveries(0).to("mock:handled").end()
+                    .onException(MyFunctionalException.class).handled(true).to("mock:handled").end()
                     .to("bean:myServiceBean")
                     .to("mock:result");
 
@@ -70,13 +64,10 @@
                 from("direct:start3")
                     // route specific error handler that is different than the global error handler
                     // here we do not redeliver and send errors to mock:error3 instead of the global endpoint
-                    .errorHandler(deadLetterChannel("mock:error3")
-                            .maximumRedeliveries(0))
+                    .errorHandler(deadLetterChannel("mock:error3").maximumRedeliveries(0))
 
                     // route specific on exception to mark MyFunctionalException as being handled
                     .onException(MyFunctionalException.class).handled(true).end()
-                    // however we want the IO exceptions to redeliver at most 3 times
-                    .onException(IOException.class).maximumRedeliveries(3).end()
                     .to("bean:myServiceBean")
                     .to("mock:result");
                 // END SNIPPET: e1

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceWithDefaultErrorHandlerTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceWithDefaultErrorHandlerTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceWithDefaultErrorHandlerTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionFromChoiceWithDefaultErrorHandlerTest.java Fri Apr  3 13:05:11 2009
@@ -16,82 +16,20 @@
  */
 package org.apache.camel.processor.onexception;
 
-import org.apache.camel.ContextTestSupport;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.impl.JndiRegistry;
 
 /**
  * Unit test for CAMEL-1188
  */
-public class OnExceptionFromChoiceTest extends ContextTestSupport {
-
-    private MyServiceBean myServiceBean;
-
-    public void testNoErrorWhen() throws Exception {
-        getMockEndpoint("mock:error").expectedMessageCount(0);
-        getMockEndpoint("mock:func").expectedMessageCount(0);
-        getMockEndpoint("mock:tech").expectedMessageCount(0);
-        getMockEndpoint("mock:otherwise").expectedMessageCount(0);
-
-        MockEndpoint mock = getMockEndpoint("mock:when");
-        mock.expectedMessageCount(1);
-
-        template.sendBody("direct:start", "<order><type>myType</type><user>James</user></order>");
-
-        assertMockEndpointsSatisfied();
-    }
-
-    public void testFunctionalError() throws Exception {
-        getMockEndpoint("mock:error").expectedMessageCount(0);
-        getMockEndpoint("mock:tech").expectedMessageCount(0);
-        getMockEndpoint("mock:when").expectedMessageCount(0);
-        getMockEndpoint("mock:otherwise").expectedMessageCount(0);
-
-        MockEndpoint mock = getMockEndpoint("mock:func");
-        mock.expectedMessageCount(1);
-
-        template.sendBody("direct:start", "<order><type>myType</type><user>Func</user></order>");
-
-        assertMockEndpointsSatisfied();
-    }
-
-    public void testTechnicalError() throws Exception {
-        getMockEndpoint("mock:error").expectedMessageCount(0);
-        getMockEndpoint("mock:func").expectedMessageCount(0);
-        getMockEndpoint("mock:when").expectedMessageCount(0);
-        getMockEndpoint("mock:otherwise").expectedMessageCount(0);
-
-        MockEndpoint mock = getMockEndpoint("mock:tech");
-        mock.expectedMessageCount(1);
-
-        template.sendBody("direct:start", "<order><type>myType</type><user>Tech</user></order>");
-
-        assertMockEndpointsSatisfied();
-    }
-
-    @Override
-    protected void setUp() throws Exception {
-        myServiceBean = new MyServiceBean();
-        super.setUp();
-    }
-
-    @Override
-    protected JndiRegistry createRegistry() throws Exception {
-        JndiRegistry jndi = super.createRegistry();
-        jndi.bind("myServiceBean", myServiceBean);
-        return jndi;
-    }
+public class OnExceptionFromChoiceWithDefaultErrorHandlerTest extends OnExceptionFromChoiceTest {
 
     @Override
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                errorHandler(deadLetterChannel("mock:error"));
-
-                onException(MyTechnicalException.class).maximumRedeliveries(0).handled(true).to("mock:tech");
-                onException(MyFunctionalException.class).maximumRedeliveries(0).handled(true).to("mock:func");
+                onException(MyTechnicalException.class).handled(true).to("mock:tech");
+                onException(MyFunctionalException.class).handled(true).to("mock:func");
 
                 from("direct:start")
                     .choice()

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformWithDefaultErrorHandlerTest.java (from r761591, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformWithDefaultErrorHandlerTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformWithDefaultErrorHandlerTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformTest.java&r1=761591&r2=761660&rev=761660&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/onexception/OnExceptionHandleAndTransformWithDefaultErrorHandlerTest.java Fri Apr  3 13:05:11 2009
@@ -24,14 +24,12 @@
 /**
  * Unit test inspired by end user
  */
-public class OnExceptionHandleAndTransformTest extends ContextTestSupport {
+public class OnExceptionHandleAndTransformWithDefaultErrorHandlerTest extends ContextTestSupport {
 
     public void testOnExceptionTransformConstant() throws Exception {
         context.addRoutes(new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(0));
-
                 // START SNIPPET: e1
                 // we catch MyFunctionalException and want to mark it as handled (= no failure returned to client)
                 // but we want to return a fixed text response, so we transform OUT body as Sorry.
@@ -56,8 +54,6 @@
         context.addRoutes(new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(0));
-
                 // START SNIPPET: e2
                 // we catch MyFunctionalException and want to mark it as handled (= no failure returned to client)
                 // but we want to return a fixed text response, so we transform OUT body and return the exception message
@@ -82,8 +78,6 @@
         context.addRoutes(new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                errorHandler(deadLetterChannel("mock:error").maximumRedeliveries(0));
-
                 // START SNIPPET: e3
                 // we catch MyFunctionalException and want to mark it as handled (= no failure returned to client)
                 // but we want to return a fixed text response, so we transform OUT body and return a nice message
@@ -105,4 +99,4 @@
         assertEquals("Error reported: Out of order - cannot process this message.", out);
     }
 
-}
+}
\ No newline at end of file