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/06/28 18:01:01 UTC

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

Author: davsclaus
Date: Mon Jun 28 16:01:00 2010
New Revision: 958616

URL: http://svn.apache.org/viewvc?rev=958616&view=rev
Log:
CAMEL-2868: try catch finally to support async routing engine. This one was a bit tougher as you got three things to juggle with in both sync and async mode.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java   (contents, props changed)
      - copied, changed from r958504, camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoneOutputExpressionNode.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryCatchFinallyTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryFinallyTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryThrowExceptionFinallyTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally2Test.java
      - copied, changed from r958502, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally3Test.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally4Test.java
      - copied, changed from r958502, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally5Test.java   (with props)
    camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncTryCatchFinallyTest.java
      - copied, changed from r958502, camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncFilterTest.java
Removed:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoneOutputExpressionNode.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ChoiceProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/FilterProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/Pipeline.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/TryProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Policy.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAfterFinallyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAroundFinallyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerProcessorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerRouteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TryProcessorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java
    camel/trunk/camel-core/src/test/resources/log4j.properties
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafterfinally.xml
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aoparoundfinally.xml

Copied: camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java (from r958504, camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoneOutputExpressionNode.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java?p2=camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoneOutputExpressionNode.java&r1=958504&r2=958616&rev=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoneOutputExpressionNode.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java Mon Jun 28 16:01:00 2010
@@ -34,21 +34,21 @@ import org.apache.camel.model.language.E
  * @version $Revision$
  */
 @XmlAccessorType(XmlAccessType.FIELD)
-public class NoneOutputExpressionNode extends ExpressionNode {
+public class NoOutputExpressionNode extends ExpressionNode {
 
-    public NoneOutputExpressionNode() {
+    public NoOutputExpressionNode() {
         super();
     }
 
-    public NoneOutputExpressionNode(ExpressionDefinition expression) {
+    public NoOutputExpressionNode(ExpressionDefinition expression) {
         super(expression);
     }
 
-    public NoneOutputExpressionNode(Expression expression) {
+    public NoOutputExpressionNode(Expression expression) {
         super(expression);
     }
 
-    public NoneOutputExpressionNode(Predicate predicate) {
+    public NoOutputExpressionNode(Predicate predicate) {
         super(predicate);
     }
 

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/model/NoOutputExpressionNode.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

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=958616&r1=958615&r2=958616&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 Mon Jun 28 16:01:00 2010
@@ -39,7 +39,7 @@ import org.apache.camel.util.concurrent.
  */
 @XmlRootElement(name = "recipientList")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class RecipientListDefinition<Type extends ProcessorDefinition> extends NoneOutputExpressionNode implements ExecutorServiceAwareDefinition<RecipientListDefinition> {
+public class RecipientListDefinition<Type extends ProcessorDefinition> extends NoOutputExpressionNode implements ExecutorServiceAwareDefinition<RecipientListDefinition> {
 
     @XmlTransient
     private AggregationStrategy aggregationStrategy;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java Mon Jun 28 16:01:00 2010
@@ -35,7 +35,7 @@ import org.apache.camel.spi.RouteContext
  */
 @XmlRootElement(name = "routingSlip")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class RoutingSlipDefinition <Type extends ProcessorDefinition> extends NoneOutputExpressionNode {
+public class RoutingSlipDefinition <Type extends ProcessorDefinition> extends NoOutputExpressionNode {
     public static final String DEFAULT_DELIMITER = ",";
     @XmlAttribute
     private String headerName;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java Mon Jun 28 16:01:00 2010
@@ -30,7 +30,7 @@ import org.apache.camel.spi.RouteContext
  */
 @XmlRootElement(name = "setBody")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class SetBodyDefinition extends NoneOutputExpressionNode {
+public class SetBodyDefinition extends NoOutputExpressionNode {
 
     public SetBodyDefinition() {
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java Mon Jun 28 16:01:00 2010
@@ -34,7 +34,7 @@ import org.apache.camel.util.ObjectHelpe
  */
 @XmlRootElement(name = "setHeader")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class SetHeaderDefinition extends NoneOutputExpressionNode {
+public class SetHeaderDefinition extends NoOutputExpressionNode {
     @XmlAttribute(required = true)
     private String headerName;
     

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java Mon Jun 28 16:01:00 2010
@@ -34,7 +34,7 @@ import org.apache.camel.util.ObjectHelpe
  */
 @XmlRootElement(name = "setProperty")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class SetPropertyDefinition extends NoneOutputExpressionNode {
+public class SetPropertyDefinition extends NoOutputExpressionNode {
     @XmlAttribute(required = true)
     private String propertyName;
     

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java Mon Jun 28 16:01:00 2010
@@ -22,7 +22,6 @@ import javax.xml.bind.annotation.XmlRoot
 
 import org.apache.camel.Expression;
 import org.apache.camel.Processor;
-import org.apache.camel.builder.ExpressionClause;
 import org.apache.camel.processor.TransformProcessor;
 import org.apache.camel.spi.RouteContext;
 
@@ -31,7 +30,7 @@ import org.apache.camel.spi.RouteContext
  */
 @XmlRootElement(name = "transform")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class TransformDefinition extends NoneOutputExpressionNode {
+public class TransformDefinition extends NoOutputExpressionNode {
 
     public TransformDefinition() {
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java Mon Jun 28 16:01:00 2010
@@ -31,7 +31,7 @@ import org.apache.camel.spi.RouteContext
  */
 @XmlRootElement(name = "validate")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class ValidateDefinition extends NoneOutputExpressionNode {
+public class ValidateDefinition extends NoOutputExpressionNode {
 
     public ValidateDefinition() {
         super();

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ChoiceProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ChoiceProcessor.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ChoiceProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ChoiceProcessor.java Mon Jun 28 16:01:00 2010
@@ -53,7 +53,20 @@ public class ChoiceProcessor extends Ser
     public boolean process(Exchange exchange, AsyncCallback callback) {
         for (FilterProcessor filterProcessor : filters) {
             Predicate predicate = filterProcessor.getPredicate();
-            if (predicate != null && predicate.matches(exchange)) {
+
+            boolean matches = false;
+            try {
+                // ensure we handle exceptions thrown when matching predicate
+                if (predicate != null) {
+                    matches = predicate.matches(exchange);
+                }
+            } catch (Throwable e) {
+                exchange.setException(e);
+                callback.done(true);
+                return true;
+            }
+
+            if (matches) {
                 // process next will also take care (has not null test) if next was a stop().
                 // stop() has no processor to execute, and thus we will end in a NPE
                 return filterProcessor.processNext(exchange, callback);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/FilterProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/FilterProcessor.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/FilterProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/FilterProcessor.java Mon Jun 28 16:01:00 2010
@@ -41,7 +41,12 @@ public class FilterProcessor extends Del
 
     @Override
     public boolean process(Exchange exchange, AsyncCallback callback) {
-        boolean matches = predicate.matches(exchange);
+        boolean matches = false;
+        try {
+            matches = predicate.matches(exchange);
+        } catch (Throwable e) {
+            exchange.setException(e);
+        }
 
         if (LOG.isDebugEnabled()) {
             LOG.debug("Filter matches: " + matches + " for exchange: " + exchange);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/Pipeline.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/Pipeline.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/Pipeline.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/Pipeline.java Mon Jun 28 16:01:00 2010
@@ -149,7 +149,7 @@ public class Pipeline extends MulticastP
 
                 // continue processing the pipeline asynchronously
                 Exchange nextExchange = exchange;
-                while (processors.hasNext()) {
+                while (continueRouting(processors, nextExchange)) {
                     AsyncProcessor processor = AsyncProcessorTypeConverter.convert(processors.next());
 
                     // check for error if so we should break out

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/TryProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/TryProcessor.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/TryProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/TryProcessor.java Mon Jun 28 16:01:00 2010
@@ -17,13 +17,18 @@
 package org.apache.camel.processor;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.List;
 
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
 import org.apache.camel.Exchange;
 import org.apache.camel.Navigate;
 import org.apache.camel.Processor;
 import org.apache.camel.impl.ServiceSupport;
-import org.apache.camel.util.ExchangeHelper;
+import org.apache.camel.impl.converter.AsyncProcessorTypeConverter;
+import org.apache.camel.util.AsyncProcessorHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,24 +38,23 @@ import org.apache.commons.logging.LogFac
  *
  * @version $Revision$
  */
-public class TryProcessor extends ServiceSupport implements Processor, Navigate<Processor>, Traceable {
+public class TryProcessor extends ServiceSupport implements AsyncProcessor, Navigate<Processor>, Traceable {
     private static final transient Log LOG = LogFactory.getLog(TryProcessor.class);
 
-    // TODO: support async routing engine
-
-    protected final Processor tryProcessor;
-    protected final List<CatchProcessor> catchClauses;
-    protected final Processor finallyProcessor;
+    protected final AsyncProcessor tryProcessor;
+    protected final DoCatchProcessor catchProcessor;
+    protected final DoFinallyProcessor finallyProcessor;
+    private List<AsyncProcessor> processors;
 
     public TryProcessor(Processor tryProcessor, List<CatchProcessor> catchClauses, Processor finallyProcessor) {
-        this.tryProcessor = tryProcessor;
-        this.catchClauses = catchClauses;
-        this.finallyProcessor = finallyProcessor;
+        this.tryProcessor = AsyncProcessorTypeConverter.convert(tryProcessor);
+        this.catchProcessor = new DoCatchProcessor(catchClauses);
+        this.finallyProcessor = new DoFinallyProcessor(finallyProcessor);
     }
 
     public String toString() {
         String finallyText = (finallyProcessor == null) ? "" : " Finally {" + finallyProcessor + "}";
-        return "Try {" + tryProcessor + "} " + (catchClauses != null ? catchClauses : "") + finallyText;
+        return "Try {" + tryProcessor + "} " + (catchProcessor != null ? catchProcessor : "") + finallyText;
     }
 
     public String getTraceLabel() {
@@ -58,96 +62,131 @@ public class TryProcessor extends Servic
     }
 
     public void process(Exchange exchange) throws Exception {
-        Exception e;
-
-        // try processor first
-        try {
-            tryProcessor.process(exchange);
-            e = exchange.getException();
-
-            // Ignore it if it was handled by the dead letter channel.
-            if (e != null && ExchangeHelper.isFailureHandled(exchange)) {
-                e = null;
-            }
-        } catch (Exception ex) {
-            e = ex;
-            exchange.setException(e);
-        }
-
-        // handle any exception occurred during the try processor
-        try {
-            if (e != null) {
-                handleException(exchange, e);
-            }
-        } finally {
-            // and run finally
-            // notice its always executed since we always enter the try block
-            processFinally(exchange);
-        }
+        AsyncProcessorHelper.process(this, exchange);
     }
 
-    protected void doStart() throws Exception {
-        ServiceHelper.startServices(tryProcessor, catchClauses, finallyProcessor);
-    }
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        Iterator<AsyncProcessor> processors = getProcessors().iterator();
 
-    protected void doStop() throws Exception {
-        ServiceHelper.stopServices(finallyProcessor, catchClauses, tryProcessor);
-    }
+        while (continueRouting(processors, exchange)) {
+            prepareResult(exchange);
 
-    protected void handleException(Exchange exchange, Throwable e) throws Exception {
-        if (catchClauses == null) {
-            return;
-        }
+            // process the next processor
+            AsyncProcessor processor = processors.next();
+            boolean sync = process(exchange, callback, processor, processors);
 
-        for (CatchProcessor catchClause : catchClauses) {
-            Throwable caught = catchClause.catches(exchange, e);
-            if (caught != null) {
+            // continue as long its being processed synchronously
+            if (!sync) {
                 if (LOG.isTraceEnabled()) {
-                    LOG.trace("This TryProcessor catches the exception: " + caught.getClass().getName() + " caused by: " + e.getMessage());
+                    LOG.trace("Processing exchangeId: " + exchange.getExchangeId() + " is continued being processed asynchronously");
                 }
+                // the remainder of the try .. catch .. finally will be completed async
+                // so we break out now, then the callback will be invoked which then continue routing from where we left here
+                return false;
+            }
 
-                // give the rest of the pipeline another chance
-                exchange.setProperty(Exchange.EXCEPTION_CAUGHT, caught);
-                exchange.setException(null);
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Processing exchangeId: " + exchange.getExchangeId() + " is continued being processed synchronously");
+            }
+        }
 
-                // do not catch any exception here, let it propagate up
-                catchClause.process(exchange);
+        prepareResult(exchange);
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Processing complete for exchangeId: " + exchange.getExchangeId() + " >>> " + exchange);
+        }
+        callback.done(true);
+        return true;
+    }
 
-                // is the exception handled by the catch clause
-                boolean handled = catchClause.handles(exchange);
+    protected boolean process(final Exchange exchange, final AsyncCallback callback,
+                              final AsyncProcessor processor, final Iterator<AsyncProcessor> processors) {
+        if (LOG.isTraceEnabled()) {
+            // this does the actual processing so log at trace level
+            LOG.trace("Processing exchangeId: " + exchange.getExchangeId() + " >>> " + exchange);
+        }
 
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("The exception is handled: " + handled + " for the exception: " + e.getClass().getName()
-                        + " caused by: " + caught.getMessage());
+        // implement asynchronous routing logic in callback so we can have the callback being
+        // triggered and then continue routing where we left
+        boolean sync = processor.process(exchange, new AsyncCallback() {
+            public void done(boolean doneSync) {
+                // we only have to handle async completion of the pipeline
+                if (doneSync) {
+                    return;
                 }
 
-                if (!handled) {
-                    // put exception back as it was not handled
-                    if (exchange.getException() == null) {
-                        exchange.setException(exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class));
+                // continue processing the try .. catch .. finally asynchronously
+                while (continueRouting(processors, exchange)) {
+                    prepareResult(exchange);
+
+                    // process the next processor
+                    AsyncProcessor processor = processors.next();
+                    doneSync = process(exchange, callback, processor, processors);
+
+                    if (!doneSync) {
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace("Processing exchangeId: " + exchange.getExchangeId() + " is continued being processed asynchronously");
+                        }
+                        // the remainder of the try .. catch .. finally will be completed async
+                        // so we break out now, then the callback will be invoked which then continue routing from where we left here
+                        return;
                     }
                 }
 
-                return;
+                prepareResult(exchange);
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Processing complete for exchangeId: " + exchange.getExchangeId() + " >>> " + exchange);
+                }
+                callback.done(false);
             }
-        }
+        });
 
-        if (LOG.isTraceEnabled()) {
-            LOG.trace("This TryProcessor does not catch the exception: " + e.getClass().getName() + " caused by: " + e.getMessage());
-        }
+        return sync;
     }
 
-    protected void processFinally(Exchange exchange) throws Exception {
-        if (finallyProcessor != null) {
-            Exception lastException = exchange.getException();
-            exchange.setException(null);
+    protected Collection<AsyncProcessor> getProcessors() {
+        return processors;
+    }
 
-            // do not catch any exception here, let it propagate up
-            finallyProcessor.process(exchange);
-            if (exchange.getException() == null) {
-                exchange.setException(lastException);
+    protected boolean continueRouting(Iterator<AsyncProcessor> it, Exchange exchange) {
+        Object stop = exchange.getProperty(Exchange.ROUTE_STOP);
+        if (stop != null) {
+            boolean doStop = exchange.getContext().getTypeConverter().convertTo(Boolean.class, stop);
+            if (doStop) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Exchange is marked to stop routing: " + exchange);
+                }
+                return false;
             }
         }
+
+        // continue if there are more processors to route
+        return it.hasNext();
+    }
+
+    /**
+     * Strategy to prepare results before next iterator or when we are complete
+     *
+     * @param exchange the result exchange
+     */
+    protected static void prepareResult(Exchange exchange) {
+        // we are routing using pipes and filters so we need to manually copy OUT to IN
+        if (exchange.hasOut()) {
+            exchange.getIn().copyFrom(exchange.getOut());
+            exchange.setOut(null);
+        }
+    }
+
+    protected void doStart() throws Exception {
+        processors = new ArrayList<AsyncProcessor>();
+        processors.add(tryProcessor);
+        processors.add(catchProcessor);
+        processors.add(finallyProcessor);
+        ServiceHelper.startServices(tryProcessor, catchProcessor, finallyProcessor);
+    }
+
+    protected void doStop() throws Exception {
+        ServiceHelper.stopServices(finallyProcessor, catchProcessor, tryProcessor);
+        processors.clear();
     }
 
     public List<Processor> next() {
@@ -158,8 +197,8 @@ public class TryProcessor extends Servic
         if (tryProcessor != null) {
             answer.add(tryProcessor);
         }
-        if (catchClauses != null) {
-            answer.addAll(catchClauses);
+        if (catchProcessor != null) {
+            answer.add(catchProcessor);
         }
         if (finallyProcessor != null) {
             answer.add(finallyProcessor);
@@ -170,4 +209,208 @@ public class TryProcessor extends Servic
     public boolean hasNext() {
         return tryProcessor != null;
     }
+
+    /**
+     * Processor to handle do catch supporting asynchronous routing engine
+     */
+    private final class DoCatchProcessor extends ServiceSupport implements AsyncProcessor, Navigate<Processor>, Traceable {
+
+        private final List<CatchProcessor> catchClauses;
+
+        private DoCatchProcessor(List<CatchProcessor> catchClauses) {
+            this.catchClauses = catchClauses;
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            AsyncProcessorHelper.process(this, exchange);
+        }
+
+        public boolean process(final Exchange exchange, final AsyncCallback callback) {
+            Exception e = exchange.getException();
+
+            if (catchClauses == null || e == null) {
+                return true;
+            }
+
+            // find a catch clause to use
+            CatchProcessor processor = null;
+            for (CatchProcessor catchClause : catchClauses) {
+                Throwable caught = catchClause.catches(exchange, e);
+                if (caught != null) {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("This TryProcessor catches the exception: " + caught.getClass().getName() + " caused by: " + e.getMessage());
+                    }
+                    processor = catchClause;
+                    break;
+                }
+            }
+
+            if (processor != null) {
+                // create the handle processor which performs the actual logic
+                // this processor just lookup the right catch clause to use and then let the
+                // HandleDoCatchProcessor do all the hard work (separate of concerns)
+                HandleDoCatchProcessor cool = new HandleDoCatchProcessor(processor);
+                return cool.process(exchange, callback);
+            } else {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("This TryProcessor does not catch the exception: " + e.getClass().getName() + " caused by: " + e.getMessage());
+                }
+            }
+
+            return true;
+        }
+
+        @Override
+        protected void doStart() throws Exception {
+            ServiceHelper.startService(catchClauses);
+        }
+
+        @Override
+        protected void doStop() throws Exception {
+            ServiceHelper.stopServices(catchClauses);
+        }
+
+        @Override
+        public String toString() {
+            return "Catches{" + catchClauses + "}";
+        }
+
+        public String getTraceLabel() {
+            return "doCatch";
+        }
+
+        public List<Processor> next() {
+            List<Processor> answer = new ArrayList<Processor>();
+            if (catchProcessor != null) {
+                answer.addAll(catchClauses);
+            }
+            return answer;
+        }
+
+        public boolean hasNext() {
+            return catchClauses != null && catchClauses.size() > 0;
+        }
+    }
+
+    /**
+     * Processor to handle do finally supporting asynchronous routing engine
+     */
+    private final class DoFinallyProcessor extends DelegateAsyncProcessor implements Traceable {
+
+        private DoFinallyProcessor(Processor processor) {
+            super(processor);
+        }
+
+        @Override
+        protected boolean processNext(final Exchange exchange, final AsyncCallback callback) {
+            // clear exception so finally block can be executed
+            final Exception e = exchange.getException();
+            exchange.setException(null);
+
+            boolean sync = super.processNext(exchange, new AsyncCallback() {
+                public void done(boolean doneSync) {
+                    // we only have to handle async completion of the pipeline
+                    if (doneSync) {
+                        return;
+                    }
+
+                    // set exception back on exchange
+                    if (e != null) {
+                        exchange.setException(e);
+                        exchange.setProperty(Exchange.EXCEPTION_CAUGHT, e);
+                    }
+
+                    // signal callback to continue routing async
+                    prepareResult(exchange);
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("Processing complete for exchangeId: " + exchange.getExchangeId() + " >>> " + exchange);
+                    }
+                    callback.done(false);
+                }
+            });
+
+            if (sync) {
+                // set exception back on exchange
+                if (e != null) {
+                    exchange.setException(e);
+                    exchange.setProperty(Exchange.EXCEPTION_CAUGHT, e);
+                }
+            }
+
+            return sync;
+        }
+
+        @Override
+        public String toString() {
+            return "Finally{" + getProcessor() + "}";
+        }
+
+        public String getTraceLabel() {
+            return "doFinally";
+        }
+    }
+
+    /**
+     * Processor to handle do catch supporting asynchronous routing engine
+     */
+    private final class HandleDoCatchProcessor extends DelegateAsyncProcessor {
+
+        private final CatchProcessor catchClause;
+
+        private HandleDoCatchProcessor(CatchProcessor processor) {
+            super(processor);
+            this.catchClause = processor;
+        }
+
+        @Override
+        protected boolean processNext(final Exchange exchange, final AsyncCallback callback) {
+            final Exception caught = exchange.getException();
+            if (caught == null) {
+                return true;
+            }
+
+            // give the rest of the pipeline another chance
+            exchange.setProperty(Exchange.EXCEPTION_CAUGHT, caught);
+            exchange.setException(null);
+
+            // is the exception handled by the catch clause
+            final Boolean handled = catchClause.handles(exchange);
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("The exception is handled: " + handled + " for the exception: " + caught.getClass().getName()
+                    + " caused by: " + caught.getMessage());
+            }
+
+            boolean sync = super.processNext(exchange, new AsyncCallback() {
+                public void done(boolean doneSync) {
+                    // we only have to handle async completion of the pipeline
+                    if (doneSync) {
+                        return;
+                    }
+
+                    if (!handled) {
+                        if (exchange.getException() == null) {
+                            exchange.setException(exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class));
+                        }
+                    }
+
+                    // signal callback to continue routing async
+                    prepareResult(exchange);
+                    callback.done(false);
+                }
+            });
+
+            if (sync) {
+                // set exception back on exchange
+                if (!handled) {
+                    if (exchange.getException() == null) {
+                        exchange.setException(exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class));
+                    }
+                }
+            }
+
+            return sync;
+        }
+    }
+
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Policy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Policy.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Policy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Policy.java Mon Jun 28 16:01:00 2010
@@ -25,6 +25,9 @@ import org.apache.camel.Processor;
  */
 public interface Policy {
 
+    // TODO: Should support async routing engine
+    // we may need the same bridge as we do with InterceptStrategy in DefaultChannel
+
     /**
      * Wraps any applicable interceptors around the given processor
      *

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAfterFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAfterFinallyTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAfterFinallyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAfterFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -27,7 +27,7 @@ import org.apache.camel.component.mock.M
 public class AOPAfterFinallyTest extends ContextTestSupport {
 
     public void testAOPAfterFinally() throws Exception {
-        getMockEndpoint("mock:after").message(0).outBody().isEqualTo("Bye World");
+        getMockEndpoint("mock:after").message(0).body().isEqualTo("Bye World");
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedBodiesReceived("Bye World");
 
@@ -38,10 +38,10 @@ public class AOPAfterFinallyTest extends
     }
 
     public void testAOPAfterFinallyWithException() throws Exception {
-        getMockEndpoint("mock:after").message(0).outBody().isEqualTo("Kabom the World");
+        getMockEndpoint("mock:after").message(0).body().isEqualTo("Kaboom the World");
 
         try {
-            template.requestBody("direct:start", "Kabom", String.class);
+            template.requestBody("direct:start", "Kaboom", String.class);
             fail("Should have thrown an exception");
         } catch (CamelExecutionException e) {
             assertIsInstanceOf(IllegalArgumentException.class, e.getCause());
@@ -62,7 +62,7 @@ public class AOPAfterFinallyTest extends
                         .when(body().isEqualTo("Hello World"))
                             .transform(constant("Bye World"))
                         .otherwise()
-                            .transform(constant("Kabom the World"))
+                            .transform(constant("Kaboom the World"))
                             .throwException(new IllegalArgumentException("Damn"))
                         .end()
                     .to("mock:result");

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAroundFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAroundFinallyTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAroundFinallyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/AOPAroundFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -28,7 +28,7 @@ public class AOPAroundFinallyTest extend
 
     public void testAOPAroundFinally() throws Exception {
         getMockEndpoint("mock:before").expectedBodiesReceived("Hello World");
-        getMockEndpoint("mock:after").message(0).outBody().isEqualTo("Bye World");
+        getMockEndpoint("mock:after").message(0).body().isEqualTo("Bye World");
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedBodiesReceived("Bye World");
 
@@ -39,11 +39,11 @@ public class AOPAroundFinallyTest extend
     }
 
     public void testAOPAroundFinallyWithException() throws Exception {
-        getMockEndpoint("mock:before").expectedBodiesReceived("Kabom");
-        getMockEndpoint("mock:after").message(0).outBody().isEqualTo("Kabom the World");
+        getMockEndpoint("mock:before").expectedBodiesReceived("Kaboom");
+        getMockEndpoint("mock:after").message(0).body().isEqualTo("Kaboom the World");
         
         try {
-            template.requestBody("direct:start", "Kabom", String.class);
+            template.requestBody("direct:start", "Kaboom", String.class);
             fail("Should have thrown an exception");
         } catch (CamelExecutionException e) {
             assertIsInstanceOf(IllegalArgumentException.class, e.getCause());
@@ -64,7 +64,7 @@ public class AOPAroundFinallyTest extend
                         .when(body().isEqualTo("Hello World"))
                             .transform(constant("Bye World"))
                         .otherwise()
-                            .transform(constant("Kabom the World"))
+                            .transform(constant("Kaboom the World"))
                             .throwException(new IllegalArgumentException("Damn"))
                         .end()
                     .to("mock:result");

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerProcessorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerProcessorTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerProcessorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerProcessorTest.java Mon Jun 28 16:01:00 2010
@@ -33,11 +33,11 @@ public class PolicyPerProcessorTest exte
         getMockEndpoint("mock:foo").expectedMessageCount(1);
         getMockEndpoint("mock:foo").expectedHeaderReceived("foo", "was wrapped");
         getMockEndpoint("mock:bar").expectedMessageCount(1);
-        getMockEndpoint("mock:bar").expectedHeaderReceived("foo", "police finished excution");
+        getMockEndpoint("mock:bar").expectedHeaderReceived("foo", "police finished execution");
         getMockEndpoint("mock:bar").expectedHeaderReceived("bar", "was wrapped");
         getMockEndpoint("mock:result").expectedMessageCount(1);
-        getMockEndpoint("mock:result").expectedHeaderReceived("foo", "police finished excution");               
-        getMockEndpoint("mock:result").expectedHeaderReceived("bar", "police finished excution");
+        getMockEndpoint("mock:result").expectedHeaderReceived("foo", "police finished execution");
+        getMockEndpoint("mock:result").expectedHeaderReceived("bar", "police finished execution");
 
         template.sendBody("direct:start", "Hello World");
 
@@ -93,7 +93,7 @@ public class PolicyPerProcessorTest exte
                     exchange.getIn().setHeader(name, "was wrapped");
                     // let the original processor continue routing
                     processor.process(exchange);
-                    exchange.getIn().setHeader(name, "police finished excution");
+                    exchange.getIn().setHeader(name, "police finished execution");
                 }
             };
         }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerRouteTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/PolicyPerRouteTest.java Mon Jun 28 16:01:00 2010
@@ -38,7 +38,7 @@ public class PolicyPerRouteTest extends 
         getMockEndpoint("mock:result").expectedHeaderReceived("foo", "was wrapped");
         
         getMockEndpoint("mock:response").expectedMessageCount(1);
-        getMockEndpoint("mock:response").expectedHeaderReceived("foo", "policy finished excution");
+        getMockEndpoint("mock:response").expectedHeaderReceived("foo", "policy finished execution");
         template.sendBody("direct:send", "Hello World");
 
         assertMockEndpointsSatisfied();
@@ -92,7 +92,7 @@ public class PolicyPerRouteTest extends 
                     // let the original processor continue routing
                     exchange.getIn().setHeader(name, "was wrapped");
                     processor.process(exchange);
-                    exchange.getIn().setHeader(name, "policy finished excution");
+                    exchange.getIn().setHeader(name, "policy finished execution");
                 }
             };
         }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryCatchFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryCatchFinallyTest.java?rev=958616&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryCatchFinallyTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryCatchFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -0,0 +1,56 @@
+/**
+ * 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 SimpleTryCatchFinallyTest extends ContextTestSupport {
+
+    public void testSimpleTryThrowExceptionFinally() throws Exception {
+        getMockEndpoint("mock:try").expectedMessageCount(1);
+        getMockEndpoint("mock:catch").expectedMessageCount(1);
+        getMockEndpoint("mock:finally").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .doTry()
+                        .to("mock:try")
+                        .throwException(new IllegalArgumentException("Damn"))
+                    .doCatch(IllegalArgumentException.class)
+                        .to("mock:catch")
+                    .doFinally()
+                        .to("mock:finally")
+                    .end()
+                    .to("mock:result");
+            }
+        };
+    }
+}
\ No newline at end of file

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryFinallyTest.java?rev=958616&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryFinallyTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -0,0 +1,52 @@
+/**
+ * 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 SimpleTryFinallyTest extends ContextTestSupport {
+
+    public void testSimpleTryFinally() throws Exception {
+        getMockEndpoint("mock:try").expectedMessageCount(1);
+        getMockEndpoint("mock:finally").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .doTry()
+                        .to("mock:try")
+                    .doFinally()
+                        .to("mock:finally")
+                    .end()
+                    .to("mock:result");
+            }
+        };
+    }
+}

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryThrowExceptionFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryThrowExceptionFinallyTest.java?rev=958616&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryThrowExceptionFinallyTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/SimpleTryThrowExceptionFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -0,0 +1,61 @@
+/**
+ * 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.CamelExecutionException;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class SimpleTryThrowExceptionFinallyTest extends ContextTestSupport {
+
+    public void testSimpleTryThrowExceptionFinally() throws Exception {
+        getMockEndpoint("mock:try").expectedMessageCount(1);
+        // finally should be executed
+        getMockEndpoint("mock:finally").expectedMessageCount(1);
+        // no message arrives to result
+        getMockEndpoint("mock:result").expectedMessageCount(0);
+
+        try {
+            template.sendBody("direct:start", "Hello World");
+            fail("Should have thrown exception");
+        } catch (CamelExecutionException e) {
+            assertEquals("Damn", e.getCause().getMessage());
+        }
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .doTry()
+                        .to("mock:try")
+                        .throwException(new IllegalArgumentException("Damn"))
+                    .doFinally()
+                        .to("mock:finally")
+                    .end()
+                    .to("mock:result");
+            }
+        };
+    }
+}
\ No newline at end of file

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

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

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TryProcessorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TryProcessorTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TryProcessorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TryProcessorTest.java Mon Jun 28 16:01:00 2010
@@ -125,8 +125,9 @@ public class TryProcessorTest extends Co
             assertNotNull("There should be an exception", e);
             
             // If we handle CamelException it is what we should have as an exception caught
-            assertTrue(e instanceof CamelException);
-            assertEquals("Force to fail", e.getMessage());
+            CamelException cause = assertIsInstanceOf(CamelException.class, e.getCause());
+            assertNotNull(cause);
+            assertEquals("Force to fail", cause.getMessage());
         }
     }
 

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally2Test.java (from r958502, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally2Test.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally2Test.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java&r1=958502&r2=958616&rev=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally2Test.java Mon Jun 28 16:01:00 2010
@@ -24,24 +24,26 @@ import org.apache.camel.builder.RouteBui
 /**
  * @version $Revision$
  */
-public class AsyncEndpointTryCatchFinallyTest extends ContextTestSupport {
+public class AsyncEndpointTryCatchFinally2Test extends ContextTestSupport {
 
     private static String beforeThreadName;
+    private static String middleThreadName;
     private static String afterThreadName;
 
     public void testAsyncEndpoint() throws Exception {
-        // TODO: try catch finally need to support async routing engine
-
         getMockEndpoint("mock:before").expectedBodiesReceived("Hello Camel");
-        //getMockEndpoint("mock:after").expectedBodiesReceived("Bye Camel");
-        //getMockEndpoint("mock:result").expectedBodiesReceived("Bye Camel");
+        getMockEndpoint("mock:catch").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:after").expectedBodiesReceived("Bye World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
 
         String reply = template.requestBody("direct:start", "Hello Camel", String.class);
-        //assertEquals("Bye Camel", reply);
+        assertEquals("Bye World", reply);
 
         assertMockEndpointsSatisfied();
 
-        //assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(afterThreadName));
+        assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(middleThreadName));
+        assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(afterThreadName));
+        assertFalse("Should use different threads", middleThreadName.equalsIgnoreCase(afterThreadName));
     }
 
     @Override
@@ -62,12 +64,21 @@ public class AsyncEndpointTryCatchFinall
                             })
                             .to("async:Bye Camel?failFirstAttempts=1")
                         .doCatch(Exception.class)
+                            .to("log:catch")
+                            .to("mock:catch")
                             .process(new Processor() {
                                 public void process(Exchange exchange) throws Exception {
-                                    afterThreadName = Thread.currentThread().getName();
+                                    middleThreadName = Thread.currentThread().getName();
                                 }
                             })
+                            .to("async:Bye World")
                         .doFinally()
+                            .to("log:finally")
+                            .process(new Processor() {
+                                public void process(Exchange exchange) throws Exception {
+                                    afterThreadName = Thread.currentThread().getName();
+                                }
+                            })
                             .to("log:after")
                             .to("mock:after")
                         .end()

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally3Test.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally3Test.java?rev=958616&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally3Test.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally3Test.java Mon Jun 28 16:01:00 2010
@@ -0,0 +1,104 @@
+/**
+ * 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.async;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class AsyncEndpointTryCatchFinally3Test extends ContextTestSupport {
+
+    private static String beforeThreadName;
+    private static String middleThreadName;
+    private static String afterThreadName;
+    private static String resultThreadName;
+
+    public void testAsyncEndpoint() throws Exception {
+        getMockEndpoint("mock:before").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:catch").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:after").expectedBodiesReceived("Bye World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye Camel");
+
+        String reply = template.requestBody("direct:start", "Hello Camel", String.class);
+        assertEquals("Bye Camel", reply);
+
+        assertMockEndpointsSatisfied();
+
+        Set<String> names = new HashSet<String>();
+        names.add(beforeThreadName);
+        names.add(middleThreadName);
+        names.add(afterThreadName);
+        names.add(resultThreadName);
+
+        assertEquals("Should use 4 different threads", 4, names.size());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                context.addComponent("async", new MyAsyncComponent());
+
+                from("direct:start")
+                        .to("mock:before")
+                        .to("log:before")
+                        .doTry()
+                            .process(new Processor() {
+                                public void process(Exchange exchange) throws Exception {
+                                    beforeThreadName = Thread.currentThread().getName();
+                                }
+                            })
+                            .to("async:Bye Camel?failFirstAttempts=1")
+                        .doCatch(Exception.class)
+                            .to("log:catch")
+                            .to("mock:catch")
+                            .process(new Processor() {
+                                public void process(Exchange exchange) throws Exception {
+                                    middleThreadName = Thread.currentThread().getName();
+                                }
+                            })
+                            .to("async:Bye World")
+                        .doFinally()
+                            .process(new Processor() {
+                                public void process(Exchange exchange) throws Exception {
+                                    afterThreadName = Thread.currentThread().getName();
+                                }
+                            })
+                            .to("log:after")
+                            .to("mock:after")
+                            .to("async:Bye Camel")
+                        .end()
+                        .process(new Processor() {
+                            public void process(Exchange exchange) throws Exception {
+                                resultThreadName = Thread.currentThread().getName();
+                            }
+                        })
+                        .to("log:result")
+                        .to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

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

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

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally4Test.java (from r958502, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally4Test.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally4Test.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java&r1=958502&r2=958616&rev=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally4Test.java Mon Jun 28 16:01:00 2010
@@ -17,31 +17,23 @@
 package org.apache.camel.processor.async;
 
 import org.apache.camel.ContextTestSupport;
-import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
 
 /**
  * @version $Revision$
  */
-public class AsyncEndpointTryCatchFinallyTest extends ContextTestSupport {
-
-    private static String beforeThreadName;
-    private static String afterThreadName;
+public class AsyncEndpointTryCatchFinally4Test extends ContextTestSupport {
 
     public void testAsyncEndpoint() throws Exception {
-        // TODO: try catch finally need to support async routing engine
-
-        getMockEndpoint("mock:before").expectedBodiesReceived("Hello Camel");
-        //getMockEndpoint("mock:after").expectedBodiesReceived("Bye Camel");
-        //getMockEndpoint("mock:result").expectedBodiesReceived("Bye Camel");
+        getMockEndpoint("mock:try").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:catch").expectedMessageCount(0);
+        getMockEndpoint("mock:finally").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
 
         String reply = template.requestBody("direct:start", "Hello Camel", String.class);
-        //assertEquals("Bye Camel", reply);
+        assertEquals("Bye World", reply);
 
         assertMockEndpointsSatisfied();
-
-        //assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(afterThreadName));
     }
 
     @Override
@@ -52,24 +44,14 @@ public class AsyncEndpointTryCatchFinall
                 context.addComponent("async", new MyAsyncComponent());
 
                 from("direct:start")
-                        .to("mock:before")
-                        .to("log:before")
                         .doTry()
-                            .process(new Processor() {
-                                public void process(Exchange exchange) throws Exception {
-                                    beforeThreadName = Thread.currentThread().getName();
-                                }
-                            })
-                            .to("async:Bye Camel?failFirstAttempts=1")
-                        .doCatch(Exception.class)
-                            .process(new Processor() {
-                                public void process(Exchange exchange) throws Exception {
-                                    afterThreadName = Thread.currentThread().getName();
-                                }
-                            })
+                            .to("mock:try")
+                        .doCatch(IllegalArgumentException.class)
+                            .to("mock:catch")
+                            .to("async:Bye Camel")
                         .doFinally()
-                            .to("log:after")
-                            .to("mock:after")
+                            .to("mock:finally")
+                            .to("async:Bye World")
                         .end()
                         .to("mock:result");
             }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally5Test.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally5Test.java?rev=958616&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally5Test.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinally5Test.java Mon Jun 28 16:01:00 2010
@@ -0,0 +1,66 @@
+/**
+ * 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.async;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class AsyncEndpointTryCatchFinally5Test extends ContextTestSupport {
+
+    public void testAsyncEndpoint() throws Exception {
+        getMockEndpoint("mock:try").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:catch").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:finally").expectedBodiesReceived("Bye Camel");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
+
+        String reply = template.requestBody("direct:start", "Hello Camel", String.class);
+        assertEquals("Bye World", reply);
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                context.addComponent("async", new MyAsyncComponent());
+
+                from("direct:start")
+                        .doTry()
+                            .to("log:try")
+                            .to("mock:try")
+                            .throwException(new IllegalArgumentException("Damn"))
+                        .doCatch(IllegalArgumentException.class)
+                            .to("mock:catch")
+                            .to("log:catch")
+                            .to("async:Bye Camel")
+                        .doFinally()
+                            .to("mock:finally")
+                            .to("log:finally")
+                            .to("async:Bye World")
+                        .end()
+                        .to("mock:result")
+                        .to("log:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

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

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

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncEndpointTryCatchFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -30,18 +30,16 @@ public class AsyncEndpointTryCatchFinall
     private static String afterThreadName;
 
     public void testAsyncEndpoint() throws Exception {
-        // TODO: try catch finally need to support async routing engine
-
         getMockEndpoint("mock:before").expectedBodiesReceived("Hello Camel");
-        //getMockEndpoint("mock:after").expectedBodiesReceived("Bye Camel");
-        //getMockEndpoint("mock:result").expectedBodiesReceived("Bye Camel");
+        getMockEndpoint("mock:after").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
 
         String reply = template.requestBody("direct:start", "Hello Camel", String.class);
-        //assertEquals("Bye Camel", reply);
+        assertEquals("Bye World", reply);
 
         assertMockEndpointsSatisfied();
 
-        //assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(afterThreadName));
+        assertFalse("Should use different threads", beforeThreadName.equalsIgnoreCase(afterThreadName));
     }
 
     @Override
@@ -70,6 +68,7 @@ public class AsyncEndpointTryCatchFinall
                         .doFinally()
                             .to("log:after")
                             .to("mock:after")
+                            .transform(constant("Bye World"))
                         .end()
                         .to("mock:result");
             }

Modified: camel/trunk/camel-core/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/resources/log4j.properties?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/resources/log4j.properties (original)
+++ camel/trunk/camel-core/src/test/resources/log4j.properties Mon Jun 28 16:01:00 2010
@@ -32,6 +32,7 @@ log4j.logger.org.apache.activemq.spring=
 #log4j.logger.org.apache.camel.processor.RecipientList=TRACE
 #log4j.logger.org.apache.camel.processor.RecipientListProcessor=TRACE
 #log4j.logger.org.apache.camel.processor.RoutingSlip=TRACE
+#log4j.logger.org.apache.camel.processor.TryProcessor=TRACE
 #log4j.logger.org.apache.camel.processor.loadbalancer=TRACE
 log4j.logger.org.apache.camel.impl.converter=WARN
 log4j.logger.org.apache.camel.management=WARN

Copied: camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncTryCatchFinallyTest.java (from r958502, camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncFilterTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncTryCatchFinallyTest.java?p2=camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncTryCatchFinallyTest.java&p1=camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncFilterTest.java&r1=958502&r2=958616&rev=958616&view=diff
==============================================================================
--- camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncFilterTest.java (original)
+++ camel/trunk/components/camel-jetty/src/test/java/org/apache/camel/component/jetty/async/JettyAsyncTryCatchFinallyTest.java Mon Jun 28 16:01:00 2010
@@ -23,13 +23,16 @@ import org.junit.Test;
 /**
  * @version $Revision$
  */
-public class JettyAsyncFilterTest extends CamelTestSupport {
+public class JettyAsyncTryCatchFinallyTest extends CamelTestSupport {
 
     @Test
     public void testJettyAsync() throws Exception {
+        getMockEndpoint("mock:try").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:catch").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:finally").expectedBodiesReceived("Bye Camel");
         getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
 
-        String reply = template.requestBody("http://localhost:8876/myservice", "Hello Camel", String.class);
+        String reply = template.requestBody("http://localhost:8872/myservice", "Hello Camel", String.class);
         assertEquals("Bye World", reply);
 
         assertMockEndpointsSatisfied();
@@ -42,9 +45,16 @@ public class JettyAsyncFilterTest extend
             public void configure() throws Exception {
                 context.addComponent("async", new MyAsyncComponent());
 
-                from("jetty:http://localhost:8876/myservice")
+                from("jetty:http://localhost:8872/myservice")
                     .convertBodyTo(String.class)
-                    .filter(body().contains("Camel"))
+                    .doTry()
+                        .to("mock:try")
+                        .throwException(new IllegalArgumentException("Damn"))
+                    .doCatch(IllegalArgumentException.class)
+                        .to("mock:catch")
+                        .to("async:Bye Camel")
+                    .doFinally()
+                        .to("mock:finally")
                         .to("async:Bye World")
                     .end()
                     .to("mock:result");

Modified: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafterfinally.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafterfinally.xml?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafterfinally.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopafterfinally.xml Mon Jun 28 16:01:00 2010
@@ -35,7 +35,7 @@
                         <transform><constant>Bye World</constant></transform>
                     </when>
                     <otherwise>
-                        <transform><constant>Kabom the World</constant></transform>
+                        <transform><constant>Kaboom the World</constant></transform>
                         <throwException ref="myException"/>
                     </otherwise>
                 </choice>

Modified: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aoparoundfinally.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aoparoundfinally.xml?rev=958616&r1=958615&r2=958616&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aoparoundfinally.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aoparoundfinally.xml Mon Jun 28 16:01:00 2010
@@ -35,7 +35,7 @@
                         <transform><constant>Bye World</constant></transform>
                     </when>
                     <otherwise>
-                        <transform><constant>Kabom the World</constant></transform>
+                        <transform><constant>Kaboom the World</constant></transform>
                         <throwException ref="myException"/>
                     </otherwise>
                 </choice>