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/05/08 08:13:17 UTC

svn commit: r772853 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/ camel-core/src/main/java/org/apache/camel/impl/ camel-core/src/main/java/org/apache/camel/impl/converter/ camel-core/src/main/java/org/apache/camel/model/ camel-core/src/...

Author: davsclaus
Date: Fri May  8 06:13:16 2009
New Revision: 772853

URL: http://svn.apache.org/viewvc?rev=772853&view=rev
Log:
CAMEL-1572: Improvements to asynd DSL and added unit test in camel-spring.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDeadLetterChannelTest.java   (contents, props changed)
      - copied, changed from r772605, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncErrorHandlerTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDefaultErrorHandlerTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitTest.java   (contents, props changed)
      - copied, changed from r772605, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncNoWaitRouteTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.java
      - copied, changed from r772605, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringChoiceTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml
      - copied, changed from r772605, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/choice.xml
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml   (with props)
Removed:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncErrorHandlerTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncNoWaitRouteTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeExchangeException.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultProducerTemplate.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/converter/FutureTypeConverter.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/AsyncProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/ExchangeHelper.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitWithErrorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteWithErrorTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/MixedPropagationTransactedTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/SpringTransactionalClientDataSourceTransactedTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceMinimalConfigurationTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceWithOnExceptionTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientWithRollbackTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java Fri May  8 06:13:16 2009
@@ -31,7 +31,6 @@
 public interface Exchange {
 
     String ASYNC_WAIT = "CamelAsyncWait";
-    String ASYNC_WAIT_TIMEOUT = "CamelAsyncWaitTimeout";
 
     String BEAN_METHOD_NAME = "CamelBeanMethodName";
     String BEAN_HOLDER = "CamelBeanHolder";

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeExchangeException.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeExchangeException.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeExchangeException.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeExchangeException.java Fri May  8 06:13:16 2009
@@ -43,7 +43,11 @@
     }
 
     protected static String createMessage(String message, Exchange exchange) {
-        return message + " on the exchange: " + exchange;
+        if (exchange != null) {
+            return message + " on the exchange: " + exchange;
+        } else {
+            return message;
+        }
     }
 
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultProducerTemplate.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultProducerTemplate.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultProducerTemplate.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultProducerTemplate.java Fri May  8 06:13:16 2009
@@ -19,7 +19,6 @@
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -564,40 +563,12 @@
         return executor.submit(task);
     }
 
-
     public <T> T asyncExtractBody(Future future, Class<T> type) {
-        try {
-            return doExtractBody(future.get(), type);
-        } catch (InterruptedException e) {
-            throw ObjectHelper.wrapRuntimeCamelException(e);
-        } catch (ExecutionException e) {
-            // execution failed due to an exception so rethrow the cause
-            throw ObjectHelper.wrapRuntimeCamelException(e.getCause());
-        }
+        return ExchangeHelper.asyncExtractBody(context, future, type);
     }
 
     public <T> T asyncExtractBody(Future future, long timeout, TimeUnit unit, Class<T> type) throws TimeoutException {
-        try {
-            if (timeout > 0) {
-                return doExtractBody(future.get(timeout, unit), type);
-            } else {
-                return doExtractBody(future.get(), type);
-            }
-        } catch (InterruptedException e) {
-            throw ObjectHelper.wrapRuntimeCamelException(e);
-        } catch (ExecutionException e) {
-            // execution failed due to an exception so rethrow the cause
-            throw ObjectHelper.wrapRuntimeCamelException(e.getCause());
-        }
-    }
-
-    private <T> T doExtractBody(Object result, Class<T> type) {
-        if (result instanceof Exchange) {
-            Exchange exchange = (Exchange) result;
-            Object answer = ExchangeHelper.extractResultBody(exchange, exchange.getPattern());
-            return context.getTypeConverter().convertTo(type, answer);
-        }
-        return context.getTypeConverter().convertTo(type, result);
+        return ExchangeHelper.asyncExtractBody(context, future, timeout, unit, type);
     }
 
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/converter/FutureTypeConverter.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/converter/FutureTypeConverter.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/converter/FutureTypeConverter.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/converter/FutureTypeConverter.java Fri May  8 06:13:16 2009
@@ -68,8 +68,11 @@
             try {
                 body = future.get();
             } catch (ExecutionException e) {
-                exchange.setException(e);
-                throw e;
+                if (e.getCause() instanceof Exception) {
+                    throw (Exception) e.getCause();
+                } else {
+                    throw e;
+                }
             }
             if (LOG.isTraceEnabled()) {
                 LOG.trace("Got future response");
@@ -79,11 +82,9 @@
                 return null;
             }
 
-            Class from = body.getClass();
-
             // maybe from is already the type we want
-            if (from.isAssignableFrom(type)) {
-                return type.cast(from);
+            if (type.isAssignableFrom(body.getClass())) {
+                return type.cast(body);
             } else if (body instanceof Exchange) {
                 Exchange result = (Exchange) body;
                 body = ExchangeHelper.extractResultBody(result, result.getPattern());

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java Fri May  8 06:13:16 2009
@@ -670,9 +670,19 @@
     }
 
     /**
-     * Breaks the route into asynchronous. The caller thread will end and the OUT message will
+     * Breaks the route into asynchronous. The caller thread will either wait for the async route
+     * to complete or imeddiately comntinue. If continue the OUT message will
      * contain a {@link java.util.concurrent.Future} handle so you can get the real response
      * later using this handle.
+     * <p/>
+     * Will default wait for the async route to complete, but this behavior can be overriden by:
+     * <ul>
+     *   <li>configuring the {@link org.apache.camel.model.AsyncDefinition#waitForTaskToComplete(boolean)}
+     * method to <tt>false</tt></li>
+     *   <li>or setting a IN message header with the key {@link org.apache.camel.Exchange#ASYNC_WAIT}
+     * to <tt>false</tt>.</li>
+     * </ul>
+     * The header will take precedence, also if its <tt>true</tt>
      *
      * @return the builder
      */
@@ -683,9 +693,17 @@
     }
 
     /**
-     * Breaks the route into asynchronous. The caller thread will end and the OUT message will
+     * Breaks the route into asynchronous. The caller thread will either wait for the async route
+     * to complete or imeddiately comntinue. If continue the OUT message will
      * contain a {@link java.util.concurrent.Future} handle so you can get the real response
      * later using this handle.
+     * <p/>
+     * Will default wait for the async route to complete, but this behavior can be overriden by:
+     * <ul>
+     *   <li>configuring the {@link org.apache.camel.model.AsyncDefinition#waitForTaskToComplete(boolean)}
+     * method to <tt>false</tt></li>
+     * </ul>
+     * The header will take precedence, also if its <tt>true</tt>
      *
      * @param poolSize the core pool size
      * @return the builder

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/AsyncProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/AsyncProcessor.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/AsyncProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/AsyncProcessor.java Fri May  8 06:13:16 2009
@@ -70,15 +70,16 @@
         // sumbit the task
         Future<Exchange> future = getExecutorService().submit(task);
 
-        // TODO: Support exchange headers for wait and timeout values, see Exchange constants
+        // compute if we should wait for task to complete or not
+        boolean wait = waitTaskComplete;
+        if (exchange.getIn().getHeader(Exchange.ASYNC_WAIT) != null) {
+            wait = exchange.getIn().getHeader(Exchange.ASYNC_WAIT, Boolean.class);
+        }
 
-        if (waitTaskComplete) {
+        if (wait) {
             // wait for task to complete
             Exchange response = future.get();
-            // if we are out capable then set the response on the original exchange
-            if (ExchangeHelper.isOutCapable(exchange)) {
-                ExchangeHelper.copyResults(exchange, response);
-            }
+            ExchangeHelper.copyResults(exchange, response);
         } else {
             // no we do not expect a reply so lets continue, set a handle to the future task
             // in case end user need it later

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DeadLetterChannel.java Fri May  8 06:13:16 2009
@@ -47,6 +47,8 @@
     // or not. Also consider MEP as InOut does not work with async then as the original caller thread
     // is expecting a reply in the sync thread.
 
+    // TODO: DLQ should handle by default, so added option to set global predicate on DLC
+
     // we can use a single shared static timer for async redeliveries
     private final Processor deadLetter;
     private final String deadLetterUri;
@@ -64,7 +66,7 @@
 
         // default behavior which can be overloaded on a per exception basis
         RedeliveryPolicy currentRedeliveryPolicy = redeliveryPolicy;
-        Processor failureProcessor = deadLetter;
+        Processor deadLetterQueue = deadLetter;
         Processor onRedeliveryProcessor = redeliveryProcessor;
     }
     
@@ -116,6 +118,7 @@
                 }
                 if (exchange.getException() == null) {
                     exchange.setException(new RejectedExecutionException());
+                    return;
                 }
             }
 
@@ -136,8 +139,9 @@
             // compute if we should redeliver or not
             boolean shouldRedeliver = shouldRedeliver(exchange, data);
             if (!shouldRedeliver) {
-                deliverToFaultProcessor(exchange, data);
-                // we should not try redeliver so we are finished
+                // no then move it to the dead letter queue
+                deliverToDeadLetterQueue(exchange, data);
+                // and we are finished since the exchanged was moved to the dead letter queue
                 return;
             }
 
@@ -249,7 +253,7 @@
             // route specific failure handler?
             Processor processor = exceptionPolicy.getErrorHandler();
             if (processor != null) {
-                data.failureProcessor = processor;
+                data.deadLetterQueue = processor;
             }
             // route specific on redelivey?
             processor = exceptionPolicy.getOnRedelivery();
@@ -287,10 +291,10 @@
     }
 
     /**
-     * All redelivery attempts failed so move the exchange to the fault processor (eg the dead letter queue)
+     * All redelivery attempts failed so move the exchange to the dead letter queue
      */
-    private void deliverToFaultProcessor(final Exchange exchange, final RedeliveryData data) {
-        if (data.failureProcessor == null) {
+    private void deliverToDeadLetterQueue(final Exchange exchange, final RedeliveryData data) {
+        if (data.deadLetterQueue == null) {
             return;
         }
 
@@ -299,26 +303,30 @@
         // must decrement the redelivery counter as we didn't process the redelivery but is
         // handling by the failure handler. So we must -1 to not let the counter be out-of-sync
         decrementRedeliveryCounter(exchange);
+        // reset cached streams so they can be read again
+        MessageHelper.resetStreamCache(exchange.getIn());
 
         try {
-            data.failureProcessor.process(exchange);
+            data.deadLetterQueue.process(exchange);
         } catch (Exception e) {
             exchange.setException(e);
         }
-        log.trace("Fault processor done");
-        prepareExchangeForFailure(exchange, data.handledPredicate);
+        log.trace("DedLetterQueue processor done");
+
+        prepareExchangeAfterMovedToDeadLetterQueue(exchange, data.handledPredicate);
 
         String msg = "Failed delivery for exchangeId: " + exchange.getExchangeId()
-                + ". Handled by the failure processor: " + data.failureProcessor;
+                + ". Moved to the dead letter queue: " + data.deadLetterQueue;
         logFailedDelivery(false, exchange, msg, data, null);
     }
 
-    private void prepareExchangeForFailure(Exchange exchange, Predicate handledPredicate) {
+    private void prepareExchangeAfterMovedToDeadLetterQueue(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.setProperty(Exchange.EXCEPTION_HANDLED, Boolean.FALSE);
             exchange.setException(exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class));
         } else {
             if (log.isDebugEnabled()) {

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/util/ExchangeHelper.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/util/ExchangeHelper.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/ExchangeHelper.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/util/ExchangeHelper.java Fri May  8 06:13:16 2009
@@ -18,6 +18,10 @@
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelExecutionException;
@@ -410,9 +414,9 @@
     public static Object extractResultBody(Exchange exchange, ExchangePattern pattern) {
         Object answer = null;
         if (exchange != null) {
-            // rethrow if there was an exception
+            // rethrow if there was an exception during execution
             if (exchange.getException() != null) {
-                throw new CamelExecutionException("Exception occured during execution ", exchange, exchange.getException());
+                throw ObjectHelper.wrapCamelExecutionException(exchange, exchange.getException());
             }
 
             // result could have a fault message
@@ -455,5 +459,67 @@
         return false;
     }
 
+    /**
+     * Extracts the body from the given future, that represents a handle to an asynchronous exchange.
+     * <p/>
+     * Will wait until the future task is complete.
+     *
+     * @param context the camel context
+     * @param future the future handle
+     * @param type the expected body response type
+     * @return the result body, can be <tt>null</tt>.
+     * @throws CamelExecutionException if the processing of the exchange failed
+     */
+    public static <T> T asyncExtractBody(CamelContext context, Future future, Class<T> type) {
+        try {
+            return doExtractBody(context, future.get(), type);
+        } catch (InterruptedException e) {
+            throw ObjectHelper.wrapRuntimeCamelException(e);
+        } catch (ExecutionException e) {
+            // execution failed due to an exception so rethrow the cause
+            throw ObjectHelper.wrapCamelExecutionException(null, e.getCause());
+        }
+    }
+
+    /**
+     * Extracts the body from the given future, that represents a handle to an asynchronous exchange.
+     * <p/>
+     * Will wait for the future task to complete, but waiting at most the timeout value.
+     *
+     * @param context the camel context
+     * @param future the future handle
+     * @param timeout timeout value
+     * @param unit    timeout unit
+     * @param type the expected body response type
+     * @return the result body, can be <tt>null</tt>.
+     * @throws CamelExecutionException if the processing of the exchange failed
+     * @throws java.util.concurrent.TimeoutException is thrown if a timeout triggered
+     */
+    public static <T> T asyncExtractBody(CamelContext context, Future future, long timeout, TimeUnit unit, Class<T> type) throws TimeoutException {
+        try {
+            if (timeout > 0) {
+                return doExtractBody(context, future.get(timeout, unit), type);
+            } else {
+                return doExtractBody(context, future.get(), type);
+            }
+        } catch (InterruptedException e) {
+            throw ObjectHelper.wrapRuntimeCamelException(e);
+        } catch (ExecutionException e) {
+            // execution failed due to an exception so rethrow the cause
+            throw ObjectHelper.wrapCamelExecutionException(null, e.getCause());
+        }
+    }
+
+    private static <T> T doExtractBody(CamelContext context, Object result, Class<T> type) {
+        if (type.isAssignableFrom(result.getClass())) {
+            return type.cast(result);
+        }
+        if (result instanceof Exchange) {
+            Exchange exchange = (Exchange) result;
+            Object answer = ExchangeHelper.extractResultBody(exchange, exchange.getPattern());
+            return context.getTypeConverter().convertTo(type, answer);
+        }
+        return context.getTypeConverter().convertTo(type, result);
+    }
 
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java Fri May  8 06:13:16 2009
@@ -35,6 +35,8 @@
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import org.apache.camel.CamelExecutionException;
+import org.apache.camel.Exchange;
 import org.apache.camel.RuntimeCamelException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -880,7 +882,7 @@
      */
     public static RuntimeCamelException wrapRuntimeCamelException(Throwable e) {
         if (e instanceof RuntimeCamelException) {
-            // don't double wrap if already a RuntimeCamelException
+            // don't double wrap
             return (RuntimeCamelException)e;
         } else {
             return new RuntimeCamelException(e);
@@ -888,6 +890,22 @@
     }
 
     /**
+     * Wraps the caused exception in a {@link CamelExecutionException} if its not
+     * already such an exception.
+     *
+     * @param e the caused exception
+     * @return the wrapper exception
+     */
+    public static CamelExecutionException wrapCamelExecutionException(Exchange exchange, Throwable e) {
+        if (e instanceof CamelExecutionException) {
+            // don't double wrap
+            return (CamelExecutionException)e;
+        } else {
+            return new CamelExecutionException("Exception occured during execution", exchange, e);
+        }
+    }
+
+    /**
      * Cleans the string to pure java identifier so we can use it for loading class names.
      * <p/>
      * Especially from Sping DSL people can have \n \t or other characters that otherwise

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDeadLetterChannelTest.java (from r772605, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncErrorHandlerTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDeadLetterChannelTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDeadLetterChannelTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncErrorHandlerTest.java&r1=772605&r2=772853&rev=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncErrorHandlerTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDeadLetterChannelTest.java Fri May  8 06:13:16 2009
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.processor.async;
 
+import org.apache.camel.CamelExecutionException;
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
@@ -27,9 +28,31 @@
  *
  * @version $Revision$
  */
-public class AsyncErrorHandlerTest extends ContextTestSupport {
+public class AsyncDeadLetterChannelTest extends ContextTestSupport {
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+
+    public void testAsyncErrorHandlerWait() throws Exception {
+        context.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                errorHandler(deadLetterChannel("mock:dead").maximumRedeliveries(2).delay(0).logStackTrace(false));
+
+                from("direct:in")
+                    .async(2)
+                    .to("mock:foo")
+                    .process(new Processor() {
+                        public void process(Exchange exchange) throws Exception {
+                            throw new Exception("Forced exception by unit test");
+                        }
+                    });
+            }
+        });
+        context.start();
 
-    public void testAsyncErrorHandler() throws Exception {
         getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
 
         MockEndpoint mock = getMockEndpoint("mock:dead");
@@ -37,19 +60,25 @@
         mock.message(0).header(Exchange.REDELIVERED).isEqualTo(Boolean.TRUE);
         mock.message(0).header(Exchange.REDELIVERY_COUNTER).isEqualTo(2);
 
-        template.sendBody("direct:in", "Hello World");
+        try {
+            template.sendBody("direct:in", "Hello World");
+            fail("Should have thrown a CamelExecutionException");
+        } catch (CamelExecutionException e) {
+            assertEquals("Forced exception by unit test", e.getCause().getMessage());
+            // expected
+        }
 
         assertMockEndpointsSatisfied();
     }
 
-    @Override
-    protected RouteBuilder createRouteBuilder() throws Exception {
-        return new RouteBuilder() {
+    public void testAsyncErrorHandlerNoWait() throws Exception {
+        context.addRoutes(new RouteBuilder() {
+            @Override
             public void configure() throws Exception {
                 errorHandler(deadLetterChannel("mock:dead").maximumRedeliveries(2).delay(0).logStackTrace(false));
 
                 from("direct:in")
-                    .async(2)
+                    .async(2).waitForTaskToComplete(false)
                     .to("mock:foo")
                     .process(new Processor() {
                         public void process(Exchange exchange) throws Exception {
@@ -57,7 +86,19 @@
                         }
                     });
             }
-        };
+        });
+        context.start();
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+
+        MockEndpoint mock = getMockEndpoint("mock:dead");
+        mock.expectedMessageCount(1);
+        mock.message(0).header(Exchange.REDELIVERED).isEqualTo(Boolean.TRUE);
+        mock.message(0).header(Exchange.REDELIVERY_COUNTER).isEqualTo(2);
+
+        template.sendBody("direct:in", "Hello World");
+
+        assertMockEndpointsSatisfied();
     }
 
 }

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDefaultErrorHandlerTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDefaultErrorHandlerTest.java?rev=772853&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDefaultErrorHandlerTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncDefaultErrorHandlerTest.java Fri May  8 06:13:16 2009
@@ -0,0 +1,90 @@
+/**
+ * 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.CamelExecutionException;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * Unit test to verify that error handling using async() also works as expected.
+ *
+ * @version $Revision$
+ */
+public class AsyncDefaultErrorHandlerTest extends ContextTestSupport {
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+
+    public void testAsyncDefaultErrorHandlerWait() throws Exception {
+        context.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:in")
+                    .async(2)
+                    .to("mock:foo")
+                    .process(new Processor() {
+                        public void process(Exchange exchange) throws Exception {
+                            throw new Exception("Forced exception by unit test");
+                        }
+                    });
+            }
+        });
+        context.start();
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+
+        try {
+            template.sendBody("direct:in", "Hello World");
+            fail("Should have thrown a CamelExecutionException");
+        } catch (CamelExecutionException e) {
+            assertEquals("Forced exception by unit test", e.getCause().getMessage());
+        }
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testAsyncDefaultErrorHandlerNoWait() throws Exception {
+        context.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:in")
+                    .async(2).waitForTaskToComplete(false)
+                    .to("mock:foo")
+                    .process(new Processor() {
+                        public void process(Exchange exchange) throws Exception {
+                            throw new Exception("Forced exception by unit test");
+                        }
+                    });
+            }
+        });
+        context.start();
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+
+        // as it turns into async and we do not wait for the task to complete
+        // we will not get notified of the exception
+        template.sendBody("direct:in", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+}
\ No newline at end of file

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

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

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitTest.java (from r772605, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncNoWaitRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncNoWaitRouteTest.java&r1=772605&r2=772853&rev=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncNoWaitRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitTest.java Fri May  8 06:13:16 2009
@@ -27,9 +27,9 @@
 /**
  * @version $Revision$
  */
-public class AsyncNoWaitRouteTest extends ContextTestSupport {
+public class AsyncRouteNoWaitTest extends ContextTestSupport {
 
-    private String route = "";
+    private static String route = "";
 
     @Override
     protected void setUp() throws Exception {
@@ -122,29 +122,36 @@
             public void configure() throws Exception {
                 // we start this route async
                 from("direct:start")
-                        // we play a bit with the message
+                            // we play a bit with the message
                         .transform(body().append(" World"))
-                                // now turn the route into async from this point forward
-                                // the caller will have a Future<Exchange> returned as response in OUT
-                                // to be used to grap the async response when he fell like it
-                                // we do not want to wait for tasks to be complete so we instruct Camel
-                                // to not wait, and therefore Camel returns the Future<Exchange> handle we
-                                // can use to get the result when we want
+                            // now turn the route into async from this point forward
+                            // the caller will have a Future<Exchange> returned as response in OUT
+                            // to be used to grap the async response when he fell like it
+                            // we do not want to wait for tasks to be complete so we instruct Camel
+                            // to not wait, and therefore Camel returns the Future<Exchange> handle we
+                            // can use to get the result when we want
                         .async().waitForTaskToComplete(false)
-                                // from this point forward this is the async route doing its work
-                                // so we do a bit of delay to simulate heavy work that takes time
+                            // from this point forward this is the async route doing its work
+                            // so we do a bit of delay to simulate heavy work that takes time
                         .to("mock:foo")
                         .delay(100)
-                                // and we also work with the message so we can prepare a response
-                        .process(new Processor() {
-                            public void process(Exchange exchange) throws Exception {
-                                route += "B";
-                                assertEquals("Hello World", exchange.getIn().getBody());
-                                exchange.getOut().setBody("Bye World");
-                            }
+                            // and we also work with the message so we can prepare a response
+                        .process(new MyProcessor())
                             // and we use mocks for unit testing
-                        }).to("mock:result");
+                        .to("mock:result");
             }
         };
     }
+
+    public static class MyProcessor implements Processor {
+
+        public MyProcessor() {
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            route += "B";
+            assertEquals("Hello World", exchange.getIn().getBody());
+            exchange.getOut().setBody("Bye World");
+        }
+    }
 }

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

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

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitWithErrorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitWithErrorTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitWithErrorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteNoWaitWithErrorTest.java Fri May  8 06:13:16 2009
@@ -29,7 +29,7 @@
  */
 public class AsyncRouteNoWaitWithErrorTest extends ContextTestSupport {
 
-    private String route = "";
+    private static String route = "";
 
     @Override
     protected void setUp() throws Exception {
@@ -98,26 +98,33 @@
             public void configure() throws Exception {
                 // we start this route async
                 from("direct:start")
-                        // we play a bit with the message
+                            // we play a bit with the message
                         .transform(body().append(" World"))
-                                // now turn the route into async from this point forward
-                                // the caller will have a Future<Exchange> returned as response in OUT
-                                // to be used to grap the async response when he fell like it
+                            // now turn the route into async from this point forward
+                            // the caller will have a Future<Exchange> returned as response in OUT
+                            // to be used to grap the async response when he fell like it
                         .async().waitForTaskToComplete(false)
-                                // from this point forward this is the async route doing its work
-                                // so we do a bit of delay to simulate heavy work that takes time
+                            // from this point forward this is the async route doing its work
+                            // so we do a bit of delay to simulate heavy work that takes time
                         .to("mock:foo")
                         .delay(100)
-                                // and we also work with the message so we can prepare a response
-                        .process(new Processor() {
-                            public void process(Exchange exchange) throws Exception {
-                                route += "B";
-                                assertEquals("Hello World", exchange.getIn().getBody());
-                                throw new IllegalArgumentException("Damn forced by unit test");
-                            }
+                            // and we also work with the message so we can prepare a response
+                        .process(new MyProcessor())
                             // and we use mocks for unit testing
-                        }).to("mock:result");
+                        .to("mock:result");
             }
         };
     }
+
+    public static class MyProcessor implements Processor {
+
+        public MyProcessor() {
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            route += "B";
+            assertEquals("Hello World", exchange.getIn().getBody());
+            throw new IllegalArgumentException("Damn forced by unit test");
+        }
+    }
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteTest.java Fri May  8 06:13:16 2009
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.processor.async;
 
-import java.util.concurrent.Future;
-
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
@@ -28,7 +26,7 @@
  */
 public class AsyncRouteTest extends ContextTestSupport {
 
-    private String route = "";
+    private static String route = "";
 
     @Override
     protected void setUp() throws Exception {
@@ -44,7 +42,7 @@
         // it will wait for the async response so we get the full response
         Object out = template.requestBody("direct:start", "Hello");
 
-        // we should run before the async processor that sets B
+        // we should not run before the async processor that sets B
         route += "A";
 
         // as it turns into a async route later we get a Future as response
@@ -85,24 +83,31 @@
                 from("direct:start")
                         // we play a bit with the message
                         .transform(body().append(" World"))
-                                // now turn the route into async from this point forward
-                                // the caller will have a Future<Exchange> returned as response in OUT
-                                // to be used to grap the async response when he fell like it
+                            // now turn the route into async from this point forward
+                            // the caller will have a Future<Exchange> returned as response in OUT
+                            // to be used to grap the async response when he fell like it
                         .async()
-                                // from this point forward this is the async route doing its work
-                                // so we do a bit of delay to simulate heavy work that takes time
+                            // from this point forward this is the async route doing its work
+                            // so we do a bit of delay to simulate heavy work that takes time
                         .to("mock:foo")
                         .delay(100)
-                                // and we also work with the message so we can prepare a response
-                        .process(new Processor() {
-                            public void process(Exchange exchange) throws Exception {
-                                route += "B";
-                                assertEquals("Hello World", exchange.getIn().getBody());
-                                exchange.getOut().setBody("Bye World");
-                            }
+                            // and we also work with the message so we can prepare a response
+                        .process(new MyProcessor())
                             // and we use mocks for unit testing
-                        }).to("mock:result");
+                        .to("mock:result");
             }
         };
     }
+
+    public static class MyProcessor implements Processor {
+
+        public MyProcessor() {
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            route += "B";
+            assertEquals("Hello World", exchange.getIn().getBody());
+            exchange.getOut().setBody("Bye World");
+        }
+    }
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteWithErrorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteWithErrorTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteWithErrorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/async/AsyncRouteWithErrorTest.java Fri May  8 06:13:16 2009
@@ -27,7 +27,7 @@
  */
 public class AsyncRouteWithErrorTest extends ContextTestSupport {
 
-    private String route = "";
+    private static String route = "";
 
     @Override
     protected void setUp() throws Exception {
@@ -84,26 +84,33 @@
             public void configure() throws Exception {
                 // we start this route async
                 from("direct:start")
-                        // we play a bit with the message
+                            // we play a bit with the message
                         .transform(body().append(" World"))
-                                // now turn the route into async from this point forward
-                                // the caller will have a Future<Exchange> returned as response in OUT
-                                // to be used to grap the async response when he fell like it
+                            // now turn the route into async from this point forward
+                            // the caller will have a Future<Exchange> returned as response in OUT
+                            // to be used to grap the async response when he fell like it
                         .async()
-                                // from this point forward this is the async route doing its work
-                                // so we do a bit of delay to simulate heavy work that takes time
+                            // from this point forward this is the async route doing its work
+                            // so we do a bit of delay to simulate heavy work that takes time
                         .to("mock:foo")
                         .delay(100)
-                                // and we also work with the message so we can prepare a response
-                        .process(new Processor() {
-                            public void process(Exchange exchange) throws Exception {
-                                route += "B";
-                                assertEquals("Hello World", exchange.getIn().getBody());
-                                throw new IllegalArgumentException("Damn forced by unit test");
-                            }
+                            // and we also work with the message so we can prepare a response
+                        .process(new MyProcessor())
                             // and we use mocks for unit testing
-                        }).to("mock:result");
+                        .to("mock:result");
             }
         };
     }
+
+    public static class MyProcessor implements Processor {
+
+        public MyProcessor() {
+        }
+
+        public void process(Exchange exchange) throws Exception {
+            route += "B";
+            assertEquals("Hello World", exchange.getIn().getBody());
+            throw new IllegalArgumentException("Damn forced by unit test");
+        }
+    }
 }
\ No newline at end of file

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/MixedPropagationTransactedTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/MixedPropagationTransactedTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/MixedPropagationTransactedTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/MixedPropagationTransactedTest.java Fri May  8 06:13:16 2009
@@ -22,6 +22,7 @@
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.spring.SpringRouteBuilder;
 import org.apache.camel.spring.SpringTestSupport;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 import org.springframework.context.support.AbstractXmlApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
 import org.springframework.jdbc.core.simple.SimpleJdbcTemplate;
@@ -98,8 +99,9 @@
             template.sendBody("direct:required", "Donkey in Action");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");
@@ -112,8 +114,9 @@
             template.sendBody("direct:new", "Donkey in Action");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");
@@ -126,8 +129,9 @@
             template.sendBody("direct:new", "Tiger in Action");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/SpringTransactionalClientDataSourceTransactedTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/SpringTransactionalClientDataSourceTransactedTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/SpringTransactionalClientDataSourceTransactedTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/SpringTransactionalClientDataSourceTransactedTest.java Fri May  8 06:13:16 2009
@@ -20,6 +20,7 @@
 
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.spring.SpringTestSupport;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 import org.springframework.context.support.AbstractXmlApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
 import org.springframework.jdbc.core.JdbcTemplate;
@@ -65,8 +66,9 @@
             template.sendBody("direct:fail", "Hello World");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceMinimalConfigurationTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceMinimalConfigurationTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceMinimalConfigurationTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceMinimalConfigurationTest.java Fri May  8 06:13:16 2009
@@ -20,6 +20,7 @@
 
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.spring.SpringTestSupport;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 import org.springframework.context.support.AbstractXmlApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
 import org.springframework.jdbc.core.JdbcTemplate;
@@ -65,8 +66,9 @@
             template.sendBody("direct:fail", "Hello World");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceTest.java Fri May  8 06:13:16 2009
@@ -23,6 +23,7 @@
 import org.apache.camel.spring.SpringRouteBuilder;
 import org.apache.camel.spring.SpringTestSupport;
 import org.apache.camel.spring.spi.SpringTransactionPolicy;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 import org.springframework.context.support.AbstractXmlApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
 import org.springframework.jdbc.core.JdbcTemplate;
@@ -80,8 +81,9 @@
             template.sendBody("direct:fail", "Hello World");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         int count = jdbc.queryForInt("select count(*) from books");

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceWithOnExceptionTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceWithOnExceptionTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceWithOnExceptionTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientDataSourceWithOnExceptionTest.java Fri May  8 06:13:16 2009
@@ -21,6 +21,7 @@
 import org.apache.camel.component.mock.MockEndpoint;
 import org.apache.camel.spring.SpringRouteBuilder;
 import org.apache.camel.spring.spi.SpringTransactionPolicy;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 
 /**
  * Unit test to demonstrate the transactional client pattern.
@@ -39,8 +40,9 @@
             template.sendBody("direct:fail", "Hello World");
         } catch (RuntimeCamelException e) {
             // expeced as we fail
-            assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertEquals("We don't have Donkeys, only Camels", e.getCause().getMessage());
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof IllegalArgumentException);
+            assertEquals("We don't have Donkeys, only Camels", e.getCause().getCause().getMessage());
         }
 
         assertMockEndpointsSatisfied();

Modified: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientWithRollbackTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientWithRollbackTest.java?rev=772853&r1=772852&r2=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientWithRollbackTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/interceptor/TransactionalClientWithRollbackTest.java Fri May  8 06:13:16 2009
@@ -24,6 +24,7 @@
 import org.apache.camel.spring.SpringRouteBuilder;
 import org.apache.camel.spring.SpringTestSupport;
 import org.apache.camel.spring.spi.SpringTransactionPolicy;
+import org.apache.camel.spring.spi.TransactedRuntimeCamelException;
 import org.springframework.context.support.AbstractXmlApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
 import org.springframework.jdbc.core.JdbcTemplate;
@@ -74,7 +75,8 @@
             template.sendBody("direct:fail", "Hello World");
             fail("Should have thrown a RollbackExchangeException");
         } catch (RuntimeCamelException e) {
-            assertTrue(e.getCause() instanceof RollbackExchangeException);
+            assertIsInstanceOf(TransactedRuntimeCamelException.class, e.getCause());
+            assertTrue(e.getCause().getCause() instanceof RollbackExchangeException);
         }
 
         int count = jdbc.queryForInt("select count(*) from books");

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java Fri May  8 06:13:16 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.spring.processor.async;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.async.AsyncRouteNoWaitTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringAsyncRouteNoWaitTest extends AsyncRouteNoWaitTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml");
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java Fri May  8 06:13:16 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.spring.processor.async;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.async.AsyncRouteNoWaitWithErrorTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringAsyncRouteNoWaitWithErrorTest extends AsyncRouteNoWaitWithErrorTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml");
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.java (from r772605, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringChoiceTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringChoiceTest.java&r1=772605&r2=772853&rev=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringChoiceTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.java Fri May  8 06:13:16 2009
@@ -14,17 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.spring.processor;
+package org.apache.camel.spring.processor.async;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.processor.ChoiceTest;
+import org.apache.camel.processor.async.AsyncRouteTest;
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
 /**
  * @version $Revision$
  */
-public class SpringChoiceTest extends ChoiceTest {
+public class SpringAsyncRouteTest extends AsyncRouteTest {
+
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/choice.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml");
     }
+
 }
\ No newline at end of file

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java Fri May  8 06:13:16 2009
@@ -0,0 +1,32 @@
+/**
+ * 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.spring.processor.async;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.async.AsyncRouteWithErrorTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringAsyncRouteWithErrorTest extends AsyncRouteWithErrorTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml");
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml Fri May  8 06:13:16 2009
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <!-- START SNIPPET: e1 -->
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <route>
+            <from uri="direct:start"/>
+            <transform>
+                <constant>Hello World</constant>
+            </transform>
+
+            <async waitForTaskToComplete="false">
+                <to uri="mock:foo"/>
+                <delay>
+                    <constant>100</constant>
+                </delay>
+                <process ref="myProcessor"/>
+                <to uri="mock:result"/>
+            </async>
+
+        </route>
+    </camelContext>
+
+    <bean id="myProcessor" class="org.apache.camel.processor.async.AsyncRouteNoWaitTest$MyProcessor"/>
+    <!-- END SNIPPET: e1 -->
+
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml Fri May  8 06:13:16 2009
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <!-- START SNIPPET: e1 -->
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <route>
+            <from uri="direct:start"/>
+            <transform>
+                <constant>Hello World</constant>
+            </transform>
+
+            <async waitForTaskToComplete="false">
+                <to uri="mock:foo"/>
+                <delay>
+                    <constant>100</constant>
+                </delay>
+                <process ref="myProcessor"/>
+                <to uri="mock:result"/>
+            </async>
+
+        </route>
+    </camelContext>
+
+    <bean id="myProcessor" class="org.apache.camel.processor.async.AsyncRouteNoWaitWithErrorTest$MyProcessor"/>
+    <!-- END SNIPPET: e1 -->
+
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteNoWaitWithErrorTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml (from r772605, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/choice.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/choice.xml&r1=772605&r2=772853&rev=772853&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/choice.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteTest.xml Fri May  8 06:13:16 2009
@@ -22,26 +22,27 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
     ">
 
-  <!-- START SNIPPET: example -->
-  <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
-    <route>
-      <from uri="direct:start"/>
-      <choice>
-        <when>
-          <xpath>$foo = 'bar'</xpath>
-          <to uri="mock:x"/>
-        </when>
-        <when>
-          <xpath>$foo = 'cheese'</xpath>
-          <to uri="mock:y"/>
-        </when>
-        <otherwise>
-          <to uri="mock:z"/>
-        </otherwise>
-      </choice>
-      <to uri="mock:end"/>
-    </route>
-  </camelContext>
-  <!-- END SNIPPET: example -->
+    <!-- START SNIPPET: e1 -->
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <route>
+            <from uri="direct:start"/>
+            <transform>
+                <constant>Hello World</constant>
+            </transform>
+
+            <async>
+                <to uri="mock:foo"/>
+                <delay>
+                    <constant>100</constant>
+                </delay>
+                <process ref="myProcessor"/>
+                <to uri="mock:result"/>
+            </async>
+
+        </route>
+    </camelContext>
+
+    <bean id="myProcessor" class="org.apache.camel.processor.async.AsyncRouteTest$MyProcessor"/>
+    <!-- END SNIPPET: e1 -->
 
 </beans>

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml?rev=772853&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml Fri May  8 06:13:16 2009
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+    <!-- START SNIPPET: e1 -->
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <route>
+            <from uri="direct:start"/>
+            <transform>
+                <constant>Hello World</constant>
+            </transform>
+
+            <async>
+                <to uri="mock:foo"/>
+                <delay>
+                    <constant>100</constant>
+                </delay>
+                <process ref="myProcessor"/>
+                <to uri="mock:result"/>
+            </async>
+
+        </route>
+    </camelContext>
+
+    <bean id="myProcessor" class="org.apache.camel.processor.async.AsyncRouteWithErrorTest$MyProcessor"/>
+    <!-- END SNIPPET: e1 -->
+
+</beans>

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/async/SpringAsyncRouteWithErrorTest.xml
------------------------------------------------------------------------------
    svn:mime-type = text/xml