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/07/29 13:53:48 UTC

svn commit: r798862 - 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/model/ camel-core/src/main/java/org/apache/camel/processor/interceptor/ camel-co...

Author: davsclaus
Date: Wed Jul 29 11:53:48 2009
New Revision: 798862

URL: http://svn.apache.org/viewvc?rev=798862&view=rev
Log:
CAMEL-1553: Fine grained configuration of runtime options such as tracer, delayer, stream cache etc.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeConfiguration.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DelayerPerRouteTest.java
      - copied, changed from r798798, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteNewConfigurationTest.java
      - copied, changed from r798774, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/HandleFaultPerRouteTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/StreamCachingPerRouteTest.java
      - copied, changed from r798798, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDelayerPerRouteTest.java
      - copied, changed from r798798, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringTracePerRouteTest.java
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringHandleFaultPerRouteTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringStreamCachingPerRouteTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/delayerperroute.xml
      - copied, changed from r798798, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/handlefaultperroute.xml   (with props)
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/streamcachingperroute.xml   (with props)
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Tracer.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ExchangeConverter.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/RouteContext.java
    camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/interceptor/delayerInterceptorTest.xml

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java Wed Jul 29 11:53:48 2009
@@ -44,7 +44,7 @@
  *
  * @version $Revision$
  */
-public interface CamelContext extends Service {
+public interface CamelContext extends Service, RuntimeConfiguration {
 
     /**
      * Gets the name of the this context.
@@ -261,6 +261,7 @@
      * Returns the converter of exchanges from one type to another
      *
      * @return the converter
+     * @deprecated
      */
     ExchangeConverter getExchangeConverter();
 
@@ -463,31 +464,4 @@
      */
     ServicePool<Endpoint, Producer> getProducerServicePool();
 
-    /**
-     * Sets whether stream caching is enabled or not (default is disabled).
-     *
-     * @param cache whether stream caching is enabled or not
-     */
-    void setStreamCaching(Boolean cache);
-
-    /**
-     * Sets whether tracing is enabled or not (default is disabled).
-     * <p/>
-     * Will use the default trace formatter.
-     * <p/>
-     * Use {@link this#addInterceptStrategy(org.apache.camel.spi.InterceptStrategy)} if you
-     * want to add a custom {@link org.apache.camel.processor.interceptor.Tracer} where you
-     * can custome the tracing options and formatting as you like.
-     *
-     * @param tracing whether tracing is enabled or not.
-     */
-    void setTracing(Boolean tracing);
-
-    /**
-     * Returns whether tracing enabled for this route.
-     *
-     * @return true if tracing is enabled
-     */
-    boolean isTracing();
-
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeConfiguration.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeConfiguration.java?rev=798862&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeConfiguration.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/RuntimeConfiguration.java Wed Jul 29 11:53:48 2009
@@ -0,0 +1,92 @@
+/**
+ * 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;
+
+/**
+ * Various runtime configuration used by {@link org.apache.camel.CamelContext} and {@link org.apache.camel.spi.RouteContext}
+ * for cross cutting functions such as tracing, delayer, stream cache and the likes.
+ *
+ * @version $Revision$
+ */
+public interface RuntimeConfiguration {
+
+    /**
+     * Sets whether stream caching is enabled or not (default is disabled).
+     * <p/>
+     * Is disabled by default
+     *
+     * @param cache whether stream caching is enabled or not
+     */
+    void setStreamCaching(Boolean cache);
+
+    /**
+     * Returns whether stream cache is enabled
+     *
+     * @return true if stream cache is enabled
+     */
+    boolean isStreamCaching();
+
+    /**
+     * Sets whether tracing is enabled or not (default is disabled).
+     * <p/>
+     * Is disabled by default
+     *
+     * @param tracing whether tracing is enabled or not.
+     */
+    void setTracing(Boolean tracing);
+
+    /**
+     * Returns whether tracing enabled
+     *
+     * @return true if tracing is enabled
+     */
+    boolean isTracing();
+
+    /**
+     * Sets whether handle fault is enabled or not (default is disabled).
+     * <p/>
+     * Is disabled by default
+     *
+     * @param handleFault whether handle fault is enabled or not.
+     */
+    void setHandleFault(Boolean handleFault);
+
+    /**
+     * Returns whether tracing enabled
+     *
+     * @return true if tracing is enabled
+     */
+    boolean isHandleFault();
+
+    /**
+     * Sets a delay value in millis that a message is delayed at every step it takes in the route path,
+     * to slow things down to better helps you to see what goes
+     * <p/>
+     * Is disabled by default
+     *
+     * @param delay delay in millis
+     */
+    void setDelayer(long delay);
+
+    /**
+     * Gets the delay value
+     *
+     * @return delay in millis, or <tt>null</tt> if disabled
+     */
+    Long getDelayer();
+
+}

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

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

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java Wed Jul 29 11:53:48 2009
@@ -73,7 +73,6 @@
 import org.apache.camel.util.ReflectionInjector;
 import static org.apache.camel.util.ServiceHelper.startServices;
 import static org.apache.camel.util.ServiceHelper.stopServices;
-import org.apache.camel.util.SystemHelper;
 import org.apache.camel.util.URISupport;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -736,28 +735,28 @@
 
     public void addInterceptStrategy(InterceptStrategy interceptStrategy) {
         getInterceptStrategies().add(interceptStrategy);
+
+        // for backwards compability or if user add them here instead of the setXXX methods
+
         if (interceptStrategy instanceof Tracer) {
             setTracing(true);
+        } else if (interceptStrategy instanceof HandleFault) {
+            setHandleFault(true);
+        } else if (interceptStrategy instanceof StreamCaching) {
+            setStreamCaching(true);
+        } else if (interceptStrategy instanceof Delayer) {
+            setDelayer(((Delayer)interceptStrategy).getDelay());
         }
     }
 
-    public boolean isStreamCacheEnabled() {
-        final Boolean value = getStreamCaching();
-        if (value != null) {
-            return value;
-        } else {
-            return SystemHelper.isSystemProperty("camel.streamCache");
-        }
+    public void setStreamCaching(Boolean cache) {
+        this.streamCache = cache;
     }
 
-    public Boolean getStreamCaching() {
+    public boolean isStreamCaching() {
         return streamCache;
     }
 
-    public void setStreamCaching(Boolean cache) {
-        this.streamCache = cache;
-    }
-
     public void setTracing(Boolean tracing) {
         this.trace = tracing;
     }
@@ -766,19 +765,7 @@
         return trace;
     }
 
-    /**
-     * Returns true if handle fault has been enabled
-     */
-    public boolean isHandleFaultEnabled() {
-        final Boolean value = getHandleFault();
-        if (value != null) {
-            return value;
-        } else {
-            return SystemHelper.isSystemProperty("camel.handleFault");
-        }
-    }
-
-    public Boolean getHandleFault() {
+    public boolean isHandleFault() {
         return handleFault;
     }
 
@@ -786,36 +773,11 @@
         this.handleFault = handleFault;
     }
 
-    /**
-     * Returns true if tracing has been enabled
-     */
-    public boolean isTraceEnabled() {
-        final Boolean value = isTracing();
-        if (value != null) {
-            return value;
-        } else {
-            return SystemHelper.isSystemProperty("camel.trace");
-        }
-    }
-
-    /**
-     * Returns the delay in millis if delaying has been enabled. Returns 0 if not enabled.
-     */
-    public long isDelayEnabled() {
-        final Long value = getDelay();
-        if (value != null) {
-            return value;
-        } else {
-            String prop = SystemHelper.getSystemProperty("camel.delay");
-            return prop != null ? Long.getLong(prop) : 0;
-        }
-    }
-
-    public Long getDelay() {
+    public Long getDelayer() {
         return delay;
     }
 
-    public void setDelay(Long delay) {
+    public void setDelayer(long delay) {
         this.delay = delay;
     }
 
@@ -877,7 +839,7 @@
 
         startServices(producerServicePool);
 
-        if (isStreamCacheEnabled()) {
+        if (isStreamCaching()) {
             // only add a new stream cache if not already configured
             if (StreamCaching.getStreamCaching(this) == null) {
                 LOG.debug("StreamCaching is enabled");
@@ -885,25 +847,16 @@
             }
         }
 
-        if (isTraceEnabled()) {
+        if (isTracing()) {
             // only add a new tracer if not already configured
-            if (Tracer.getTracer(this.getInterceptStrategies()) == null) {
+            if (Tracer.getTracer(this) == null) {
                 Tracer tracer = Tracer.createTracer(this);
                 LOG.debug("Tracing is enabled");
                 addInterceptStrategy(tracer);
             }
         }
 
-        long delayInMillis = isDelayEnabled();
-        if (delayInMillis > 0) {
-            // only add a new delayer if not already configured
-            if (Delayer.getDelayer(this) == null) {
-                LOG.debug("Delayer is enabled with: " + delayInMillis + " ms.");
-                addInterceptStrategy(new Delayer(delayInMillis));
-            }
-        }
-
-        if (isHandleFaultEnabled()) {
+        if (isHandleFault()) {
             // only add a new handle fault if not already configured
             if (HandleFault.getHandleFault(this) == null) {
                 LOG.debug("HandleFault is enabled");
@@ -911,6 +864,15 @@
             }
         }
 
+        if (getDelayer() != null && getDelayer() > 0) {
+            // only add a new delayer if not already configured
+            if (Delayer.getDelayer(this) == null) {
+                long millis = getDelayer();
+                LOG.debug("Delayer is enabled with: " + millis + " ms.");
+                addInterceptStrategy(new Delayer(millis));
+            }
+        }
+
         try {
             lifecycleStrategy.onContextStart(this);
         } catch (Exception e) {
@@ -1160,7 +1122,7 @@
 
     @Override
     public String toString() {
-        return "DefaultCamelContext(" + getName() + ")";
+        return "CamelContext(" + getName() + ")";
     }
 
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java Wed Jul 29 11:53:48 2009
@@ -50,6 +50,9 @@
     private List<InterceptStrategy> interceptStrategies = new ArrayList<InterceptStrategy>();
     private boolean routeAdded;
     private Boolean trace;
+    private Boolean stramCache;
+    private Boolean handleFault;
+    private Long delay;
 
     public DefaultRouteContext(RouteDefinition route, FromDefinition from, Collection<Route> routes) {
         this.route = route;
@@ -178,11 +181,50 @@
         if (trace != null) {
             return trace;
         } else {
-            // fallback to let the camel context decide whether tracing is enabled
+            // fallback to the option from camel context
             return getCamelContext().isTracing();
         }
     }
 
+    public void setStreamCaching(Boolean cache) {
+        this.stramCache = cache;
+    }
+
+    public boolean isStreamCaching() {
+        if (stramCache != null) {
+            return stramCache;
+        } else {
+            // fallback to the option from camel context
+            return getCamelContext().isStreamCaching();
+        }
+    }
+
+    public void setHandleFault(Boolean handleFault) {
+        this.handleFault = handleFault;
+    }
+
+    public boolean isHandleFault() {
+        if (handleFault != null) {
+            return handleFault;
+        } else {
+            // fallback to the option from camel context
+            return getCamelContext().isHandleFault();
+        }
+    }
+
+    public void setDelayer(long delay) {
+        this.delay = delay;
+    }
+
+    public Long getDelayer() {
+        if (delay != null) {
+            return delay;
+        } else {
+            // fallback to the option from camel context
+            return getCamelContext().getDelayer();
+        }
+    }
+
     public DataFormatDefinition getDataFormat(String ref) {
         Map<String, DataFormatDefinition> dataFormats = getCamelContext().getDataFormats();
         if (dataFormats != null) {

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=798862&r1=798861&r2=798862&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 Wed Jul 29 11:53:48 2009
@@ -21,9 +21,9 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -36,6 +36,7 @@
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
+import static org.apache.camel.builder.Builder.body;
 import org.apache.camel.builder.DataFormatClause;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.ErrorHandlerBuilderRef;
@@ -50,6 +51,9 @@
 import org.apache.camel.processor.Pipeline;
 import org.apache.camel.processor.aggregate.AggregationCollection;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.interceptor.Delayer;
+import org.apache.camel.processor.interceptor.HandleFault;
+import org.apache.camel.processor.interceptor.StreamCaching;
 import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.DataFormat;
@@ -61,8 +65,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import static org.apache.camel.builder.Builder.body;
-
 /**
  * Base class for processor types that most XML types extend.
  *
@@ -70,7 +72,7 @@
  */
 @XmlAccessorType(XmlAccessType.PROPERTY)
 public abstract class ProcessorDefinition<Type extends ProcessorDefinition> extends OptionalIdentifiedType<Type> implements Block {
-    private static final transient Log LOG = LogFactory.getLog(ProcessorDefinition.class);
+    protected final transient Log log = LogFactory.getLog(getClass());
     private ErrorHandlerBuilder errorHandlerBuilder;
     private NodeFactory nodeFactory;
     private final LinkedList<Block> blocks = new LinkedList<Block>();
@@ -130,12 +132,12 @@
 
             // only add regular processors as event driven
             if (endpointInterceptor) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Endpoint interceptor should not be added as an event driven consumer route: " + processor);
+                if (log.isDebugEnabled()) {
+                    log.debug("Endpoint interceptor should not be added as an event driven consumer route: " + processor);
                 }
             } else {
-                if (LOG.isTraceEnabled()) {
-                    LOG.trace("Adding event driven processor: " + processor);
+                if (log.isTraceEnabled()) {
+                    log.trace("Adding event driven processor: " + processor);
                 }
                 routeContext.addEventDrivenProcessor(processor);
             }
@@ -159,7 +161,7 @@
         Channel channel = createChannel(routeContext);
         channel.setNextProcessor(processor);
 
-        // add interceptor strategies to the channel
+        // add interceptor strategies to the channel must be in this order: camel context, route context, local
         addInterceptStrategies(routeContext, channel, routeContext.getCamelContext().getInterceptStrategies());
         addInterceptStrategies(routeContext, channel, routeContext.getInterceptStrategies());
         addInterceptStrategies(routeContext, channel, this.getInterceptStrategies());
@@ -193,6 +195,34 @@
                 // trace is disabled so we should not add it
                 continue;
             }
+            if (!routeContext.isStreamCaching() && strategy instanceof StreamCaching) {
+                // stream cache is disabled so we should not add it
+                continue;
+            }
+            if (!routeContext.isHandleFault() && strategy instanceof HandleFault) {
+                // handle fault is disabled so we should not add it
+                continue;
+            }
+            if (strategy instanceof Delayer) {
+                if ((routeContext.getDelayer() == null || routeContext.getDelayer() <= 0)) {
+                    // delayer is disabled so we should not add it
+                    continue;
+                } else {
+                    // replace existing delayer as delayer have individual configuration
+                    Iterator<InterceptStrategy> it = channel.getInterceptStrategies().iterator();
+                    while (it.hasNext()) {
+                        InterceptStrategy existing = it.next();
+                        if (existing instanceof Delayer) {
+                            it.remove();
+                        }
+                    }
+                    // add the new correct delayer
+                    channel.addInterceptStrategy(strategy);
+                    continue;
+                }
+            }
+
+            // add strategy
             channel.addInterceptStrategy(strategy);
         }
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java Wed Jul 29 11:53:48 2009
@@ -35,6 +35,8 @@
 import org.apache.camel.ServiceStatus;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.impl.DefaultRouteContext;
+import org.apache.camel.processor.interceptor.Delayer;
+import org.apache.camel.processor.interceptor.HandleFault;
 import org.apache.camel.processor.interceptor.StreamCaching;
 import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.spi.RouteContext;
@@ -55,6 +57,8 @@
     private String group;
     private Boolean streamCache;
     private Boolean trace;
+    private Boolean handleFault;
+    private Long delayer;
 
     public RouteDefinition() {
     }
@@ -235,6 +239,40 @@
         return this;
     }
 
+    /**
+     * Disable handle fault for this route.
+     */
+    public RouteDefinition noHandleFault() {
+        setHandleFault(false);
+        return this;
+    }
+
+    /**
+     * Enable handle fault for this route.
+     */
+    public RouteDefinition handleFault() {
+        setHandleFault(true);
+        return this;
+    }
+
+    /**
+     * Disable delayer for this route.
+     */
+    public RouteDefinition noDelayer() {
+        setDelayer(0L);
+        return this;
+    }
+
+    /**
+     * Enable delayer for this route.
+     *
+     * @param delay delay in millis
+     */
+    public RouteDefinition delayer(long delay) {
+        setDelayer(delay);
+        return this;
+    }
+
     // Properties
     // -----------------------------------------------------------------------
 
@@ -304,6 +342,24 @@
         this.trace = trace;
     }
 
+    public Boolean isHandleFault() {
+        return handleFault;
+    }
+
+    @XmlAttribute
+    public void setHandleFault(Boolean handleFault) {
+        this.handleFault = handleFault;
+    }
+
+    public Long getDelayer() {
+        return delayer;
+    }
+
+    @XmlAttribute
+    public void setDelayer(Long delayer) {
+        this.delayer = delayer;
+    }
+
     // Implementation methods
     // -------------------------------------------------------------------------
     protected RouteContext addRoutes(Collection<Route> routes, FromDefinition fromType) throws Exception {
@@ -313,14 +369,63 @@
         if (trace != null) {
             routeContext.setTracing(isTrace());
             if (isTrace()) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Tracing is enabled on route: " + this);
+                }
                 // only add a new tracer if not already a global configured on camel context
-                if (Tracer.getTracer(camelContext.getInterceptStrategies()) == null) {
+                if (Tracer.getTracer(camelContext) == null) {
                     Tracer tracer = Tracer.createTracer(camelContext);
                     addInterceptStrategy(tracer);
                 }
             }
         }
 
+        // configure stream caching
+        if (streamCache != null) {
+            routeContext.setStreamCaching(isStreamCache());
+            if (isStreamCache()) {
+                if (log.isDebugEnabled()) {
+                    log.debug("StramCaching is enabled on route: " + this);
+                }
+                // only add a new stream cache if not already a global configured on camel context
+                if (StreamCaching.getStreamCaching(camelContext) == null) {
+                    addInterceptStrategy(new StreamCaching());
+                }
+            }
+        }
+
+        // configure stream caching
+        if (handleFault != null) {
+            routeContext.setHandleFault(isHandleFault());
+            if (isHandleFault()) {
+                if (log.isDebugEnabled()) {
+                    log.debug("HandleFault is enabled on route: " + this);
+                }
+                // only add a new handle fault if not already a global configured on camel context
+                if (HandleFault.getHandleFault(camelContext) == null) {
+                    addInterceptStrategy(new HandleFault());
+                }
+            }
+        }
+
+        // configure delayer
+        if (delayer != null) {
+            routeContext.setDelayer(getDelayer());
+            if (getDelayer() != null) {
+                long millis = getDelayer();
+                if (millis > 0) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Delayer is enabled with: " + millis + " ms. on route: " + this);
+                    }
+                    addInterceptStrategy(new Delayer(millis));
+                } else {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Delayer is disabled on route: " + this);
+                    }
+                }
+            }
+        }
+
         // should inherit the intercept strategies we have defined
         routeContext.setInterceptStrategies(this.getInterceptStrategies());
         // force endpoint resolution

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Tracer.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Tracer.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Tracer.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Tracer.java Wed Jul 29 11:53:48 2009
@@ -67,7 +67,8 @@
      *
      * @return the tracer or null if none can be found
      */
-    public static Tracer getTracer(List<InterceptStrategy> list) {
+    public static Tracer getTracer(CamelContext context) {
+        List<InterceptStrategy> list = context.getInterceptStrategies();
         for (InterceptStrategy interceptStrategy : list) {
             if (interceptStrategy instanceof Tracer) {
                 return (Tracer)interceptStrategy;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ExchangeConverter.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ExchangeConverter.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ExchangeConverter.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ExchangeConverter.java Wed Jul 29 11:53:48 2009
@@ -22,6 +22,7 @@
  * This converter is capable of converting from an exchange to another type
  *
  * @version $Revision$
+ * @deprecated
  */
 public interface ExchangeConverter {
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/RouteContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/RouteContext.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/RouteContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/RouteContext.java Wed Jul 29 11:53:48 2009
@@ -21,8 +21,8 @@
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
-import org.apache.camel.Intercept;
 import org.apache.camel.Processor;
+import org.apache.camel.RuntimeConfiguration;
 import org.apache.camel.model.DataFormatDefinition;
 import org.apache.camel.model.FromDefinition;
 import org.apache.camel.model.ProcessorDefinition;
@@ -33,7 +33,7 @@
  *
  * @version $Revision$
  */
-public interface RouteContext {
+public interface RouteContext extends RuntimeConfiguration {
 
     /**
      * Gets the endpoint
@@ -164,18 +164,4 @@
      */
     DataFormatDefinition getDataFormat(String ref);
 
-    /**
-     * Sets whether tracing is enabled or not (default is disabled).
-     *
-     * @param tracing whether tracing is enabled or not.
-     */
-    void setTracing(Boolean tracing);
-
-    /**
-     * Returns whether tracing enabled for this route.
-     *
-     * @return true if tracing is enabled
-     */
-    boolean isTracing();
-
 }

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DelayerPerRouteTest.java (from r798798, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DelayerPerRouteTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DelayerPerRouteTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java&r1=798798&r2=798862&rev=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DelayerPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -18,28 +18,18 @@
 
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.processor.interceptor.Tracer;
 
 /**
  * @version $Revision$
  */
-public class TracePerRouteTest extends ContextTestSupport {
+public class DelayerPerRouteTest extends ContextTestSupport {
 
-    public void testTracingPerRoute() throws Exception {
-        getMockEndpoint("mock:a").expectedMessageCount(1);
-        getMockEndpoint("mock:b").expectedMessageCount(1);
-        getMockEndpoint("mock:c").expectedMessageCount(1);
-
-        // only a and c has enabled tracing
-        MockEndpoint traced = getMockEndpoint("mock:traced");
-        traced.expectedMessageCount(2);
-        traced.message(0).body(String.class).contains("mock://a");
-        traced.message(1).body(String.class).contains("mock://c");
-
-        template.sendBody("direct:a", "Hello World");
-        template.sendBody("direct:b", "Bye World");
-        template.sendBody("direct:c", "Gooday World");
+    public void testDelayerPerRoute() throws Exception {
+        getMockEndpoint("mock:result").expectedBodiesReceived("B", "A", "C");
+
+        template.sendBody("seda:a", "A");
+        template.sendBody("seda:b", "B");
+        template.sendBody("seda:c", "C");
 
         assertMockEndpointsSatisfied();
     }
@@ -49,17 +39,14 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                Tracer tracer = new Tracer();
-                tracer.setDestinationUri("mock:traced");
-                tracer.setLogName("foo");
-                context.addInterceptStrategy(tracer);
+                context.setDelayer(1000);
 
-                from("direct:a").to("mock:a");
+                from("seda:a").delayer(500).to("mock:result");
 
-                from("direct:b").noTracing().to("mock:b");
+                from("seda:b").noDelayer().to("mock:result");
 
-                from("direct:c").tracing().to("mock:c");
+                from("seda:c").to("mock:result");
             }
         };
     }
-}
+}
\ No newline at end of file

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteNewConfigurationTest.java (from r798774, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteNewConfigurationTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteNewConfigurationTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteTest.java&r1=798774&r2=798862&rev=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/FaultRetryRouteNewConfigurationTest.java Wed Jul 29 11:53:48 2009
@@ -16,54 +16,17 @@
  */
 package org.apache.camel.processor;
 
-import org.apache.camel.CamelException;
-import org.apache.camel.ContextTestSupport;
-import org.apache.camel.Exchange;
 import org.apache.camel.LoggingLevel;
-import org.apache.camel.Message;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.processor.interceptor.HandleFault;
 
-public class FaultRetryRouteTest extends ContextTestSupport {
-    protected MockEndpoint a;
-    protected MockEndpoint b;
-    protected MockEndpoint error;
+public class FaultRetryRouteNewConfigurationTest extends FaultRetryRouteTest {
 
-    protected final Processor successOnRetryProcessor = new Processor() {
-        int count;
-        public void process(Exchange exchange) throws CamelException {
-            if (count++ == 0) {
-                Message message = exchange.getFault();
-                message.setBody(new CamelException("Failed the first time"));
-            }
-        }
-    };
-
-    public void testSuccessfulRetry() throws Exception {
-        a.expectedBodiesReceived("in");
-        b.expectedBodiesReceived("in");
-        error.expectedMessageCount(0);
-
-        template.sendBody("direct:start", "in");
-
-        MockEndpoint.assertIsSatisfied(a, b, error);
-    }
-
-    @Override
-    protected void setUp() throws Exception {
-        super.setUp();
-        a = resolveMandatoryEndpoint("mock:a", MockEndpoint.class);
-        b = resolveMandatoryEndpoint("mock:b", MockEndpoint.class);
-        error = resolveMandatoryEndpoint("mock:error", MockEndpoint.class);
-    }
     @Override
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             @Override
             public void configure() {
-                getContext().addInterceptStrategy(new HandleFault());
+                getContext().setHandleFault(true);
 
                 errorHandler(
                     deadLetterChannel("mock:error")
@@ -77,4 +40,4 @@
             }
         };
     }
-}
+}
\ No newline at end of file

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/HandleFaultPerRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/HandleFaultPerRouteTest.java?rev=798862&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/HandleFaultPerRouteTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/HandleFaultPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -0,0 +1,89 @@
+/**
+ * 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.Exchange;
+import org.apache.camel.Message;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision$
+ */
+public class HandleFaultPerRouteTest extends ContextTestSupport {
+
+    public void testHandleFaultPerRoute() throws Exception {
+        MockEndpoint a = getMockEndpoint("mock:a");
+        a.expectedMessageCount(1);
+
+        MockEndpoint b = getMockEndpoint("mock:b");
+        b.expectedMessageCount(1);
+
+        MockEndpoint c = getMockEndpoint("mock:c");
+        c.expectedMessageCount(1);
+
+        Exchange outA = template.send("direct:a", new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                Message message = exchange.getFault();
+                message.setBody(new IllegalArgumentException("A"));
+            }
+        });
+        assertTrue("Should be failed", outA.isFailed());
+        assertIsInstanceOf(IllegalArgumentException.class, outA.getException());
+        assertEquals("A", outA.getException().getMessage());
+
+        Exchange outB = template.send("direct:b", new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                Message message = exchange.getFault();
+                message.setBody(new IllegalArgumentException("B"));
+            }
+        });
+
+        assertTrue("Should be failed", outB.isFailed());
+        assertNull("Should not handle fault", outB.getException());
+        assertIsInstanceOf(IllegalArgumentException.class, outB.getFault().getBody());
+
+        Exchange outC = template.send("direct:c", new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                Message message = exchange.getFault();
+                message.setBody(new IllegalArgumentException("C"));
+            }
+        });
+
+        assertTrue("Should be failed", outC.isFailed());
+        assertIsInstanceOf(IllegalArgumentException.class, outC.getException());
+        assertEquals("C", outC.getException().getMessage());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                context.setHandleFault(true);
+
+                from("direct:a").to("mock:a");
+
+                from("direct:b").noHandleFault().to("mock:b");
+
+                from("direct:c").handleFault().to("mock:c");
+            }
+        };
+    }
+}
\ No newline at end of file

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

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

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/StreamCachingPerRouteTest.java (from r798798, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/StreamCachingPerRouteTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/StreamCachingPerRouteTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java&r1=798798&r2=798862&rev=798862&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/StreamCachingPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -16,32 +16,48 @@
  */
 package org.apache.camel.processor;
 
+import java.io.StringReader;
+import javax.xml.transform.stream.StreamSource;
+
 import org.apache.camel.ContextTestSupport;
+import org.apache.camel.StreamCache;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
-import org.apache.camel.processor.interceptor.Tracer;
 
 /**
  * @version $Revision$
  */
-public class TracePerRouteTest extends ContextTestSupport {
+public class StreamCachingPerRouteTest extends ContextTestSupport {
+
+    public void testStreamCachingPerRoute() throws Exception {
+        MockEndpoint a = getMockEndpoint("mock:a");
+        a.expectedMessageCount(1);
+
+        MockEndpoint b = getMockEndpoint("mock:b");
+        b.expectedMessageCount(1);
+
+        MockEndpoint c = getMockEndpoint("mock:c");
+        c.expectedMessageCount(1);
 
-    public void testTracingPerRoute() throws Exception {
-        getMockEndpoint("mock:a").expectedMessageCount(1);
-        getMockEndpoint("mock:b").expectedMessageCount(1);
-        getMockEndpoint("mock:c").expectedMessageCount(1);
-
-        // only a and c has enabled tracing
-        MockEndpoint traced = getMockEndpoint("mock:traced");
-        traced.expectedMessageCount(2);
-        traced.message(0).body(String.class).contains("mock://a");
-        traced.message(1).body(String.class).contains("mock://c");
-
-        template.sendBody("direct:a", "Hello World");
-        template.sendBody("direct:b", "Bye World");
-        template.sendBody("direct:c", "Gooday World");
+        new StreamSource(new StringReader("A"));
+
+        template.sendBody("direct:a", new StreamSource(new StringReader("A")));
+        Object sendB = new StreamSource(new StringReader("B"));
+        template.sendBody("direct:b", sendB);
+        template.sendBody("direct:c", new StreamSource(new StringReader("C")));
 
         assertMockEndpointsSatisfied();
+
+        Object bodyA = a.getReceivedExchanges().get(0).getIn().getBody();
+        assertIsInstanceOf(StreamCache.class, bodyA);
+
+        Object bodyC = c.getReceivedExchanges().get(0).getIn().getBody();
+        assertIsInstanceOf(StreamCache.class, bodyC);
+
+        // should not be stream cache but the pure body
+        Object bodyB = b.getReceivedExchanges().get(0).getIn().getBody();
+        assertIsInstanceOf(StreamSource.class, bodyB);
+        assertSame("Should be same body as we send", sendB, bodyB);
     }
 
     @Override
@@ -49,17 +65,14 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                Tracer tracer = new Tracer();
-                tracer.setDestinationUri("mock:traced");
-                tracer.setLogName("foo");
-                context.addInterceptStrategy(tracer);
+                context.setStreamCaching(true);
 
                 from("direct:a").to("mock:a");
 
-                from("direct:b").noTracing().to("mock:b");
+                from("direct:b").noStreamCaching().to("mock:b");
 
-                from("direct:c").tracing().to("mock:c");
+                from("direct:c").streamCaching().to("mock:c");
             }
         };
     }
-}
+}
\ No newline at end of file

Modified: camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java (original)
+++ camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java Wed Jul 29 11:53:48 2009
@@ -94,7 +94,7 @@
     @XmlAttribute(required = false)
     private Boolean streamCache = Boolean.TRUE;
     @XmlAttribute(required = false)
-    private Long delay;
+    private Long delayer;
     @XmlAttribute(required = false)
     private Boolean handleFault;
     @XmlAttribute(required = false)
@@ -628,12 +628,12 @@
         this.streamCache = streamCache;
     }
 
-    public Long getDelay() {
-        return delay;
+    public Long getDelayer() {
+        return delayer;
     }
 
-    public void setDelay(Long delay) {
-        this.delay = delay;
+    public void setDelayer(Long delayer) {
+        this.delayer = delayer;
     }
 
     public Boolean getHandleFault() {
@@ -716,8 +716,8 @@
         if (trace != null) {
             ctx.setTracing(trace);
         }
-        if (delay != null) {
-            ctx.setDelay(delay);
+        if (delayer != null) {
+            ctx.setDelayer(delayer);
         }
         if (handleFault != null) {
             ctx.setHandleFault(handleFault);

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDelayerPerRouteTest.java (from r798798, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringTracePerRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDelayerPerRouteTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDelayerPerRouteTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringTracePerRouteTest.java&r1=798798&r2=798862&rev=798862&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringTracePerRouteTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringDelayerPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -17,15 +17,15 @@
 package org.apache.camel.spring.processor;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.processor.TracePerRouteTest;
+import org.apache.camel.processor.DelayerPerRouteTest;
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
 /**
  * @version $Revision$
  */
-public class SpringTracePerRouteTest extends TracePerRouteTest {
+public class SpringDelayerPerRouteTest extends DelayerPerRouteTest {
 
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/traceperroute.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/delayerperroute.xml");
     }
 }
\ No newline at end of file

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringHandleFaultPerRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringHandleFaultPerRouteTest.java?rev=798862&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringHandleFaultPerRouteTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringHandleFaultPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.HandleFaultPerRouteTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringHandleFaultPerRouteTest extends HandleFaultPerRouteTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/handlefaultperroute.xml");
+    }
+}
\ No newline at end of file

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

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

Added: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringStreamCachingPerRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringStreamCachingPerRouteTest.java?rev=798862&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringStreamCachingPerRouteTest.java (added)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringStreamCachingPerRouteTest.java Wed Jul 29 11:53:48 2009
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.StreamCachingPerRouteTest;
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+/**
+ * @version $Revision$
+ */
+public class SpringStreamCachingPerRouteTest extends StreamCachingPerRouteTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/streamcachingperroute.xml");
+    }
+}
\ No newline at end of file

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

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

Modified: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/interceptor/delayerInterceptorTest.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/interceptor/delayerInterceptorTest.xml?rev=798862&r1=798861&r2=798862&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/interceptor/delayerInterceptorTest.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/interceptor/delayerInterceptorTest.xml Wed Jul 29 11:53:48 2009
@@ -22,7 +22,7 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd 
     ">
 
-    <camelContext id="camel" delay="200" xmlns="http://camel.apache.org/schema/spring">
+    <camelContext id="camel" delayer="200" xmlns="http://camel.apache.org/schema/spring">
         <route>
             <from uri="direct:start"/>
             <to uri="mock:result"/>

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/delayerperroute.xml (from r798798, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/delayerperroute.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/delayerperroute.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml&r1=798798&r2=798862&rev=798862&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/delayerperroute.xml Wed Jul 29 11:53:48 2009
@@ -22,24 +22,27 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd
     ">
 
-    <bean id="myTracer" class="org.apache.camel.processor.interceptor.Tracer">
-        <property name="destinationUri" value="mock:traced"/>
-        <property name="logName" value="foo"/>
-    </bean>
+    <!-- define a global delayer using 1000 ms -->
+    <camelContext id="camel" delayer="1000" xmlns="http://camel.apache.org/schema/spring">
 
-    <camelContext id="camel" trace="true" xmlns="http://camel.apache.org/schema/spring">
-        <route>
-            <from uri="direct:a"/>
-            <to uri="mock:a"/>
+        <!-- this route uses a 500 ms delayer -->
+        <route delayer="500">
+            <from uri="seda:a"/>
+            <to uri="mock:result"/>
         </route>
-        <route trace="false">
-            <from uri="direct:b"/>
-            <to uri="mock:b"/>
+
+        <!-- use -1 to indicate delayer is disabled on this particular route (you can also use 0 to disable) -->
+        <route delayer="-1">
+            <from uri="seda:b"/>
+            <to uri="mock:result"/>
         </route>
-        <route trace="true">
-            <from uri="direct:c"/>
-            <to uri="mock:c"/>
+
+        <!-- this route will inherit the global delayer -->
+        <route>
+            <from uri="seda:c"/>
+            <to uri="mock:result"/>
         </route>
+
     </camelContext>
 
 </beans>

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/handlefaultperroute.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/handlefaultperroute.xml?rev=798862&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/handlefaultperroute.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/handlefaultperroute.xml Wed Jul 29 11:53:48 2009
@@ -0,0 +1,44 @@
+<?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
+    ">
+
+    <camelContext id="camel" handleFault="true" xmlns="http://camel.apache.org/schema/spring">
+
+        <route handleFault="true">
+            <from uri="direct:a"/>
+            <to uri="mock:a"/>
+        </route>
+
+        <route handleFault="false">
+            <from uri="direct:b"/>
+            <to uri="mock:b"/>
+        </route>
+
+        <route>
+            <from uri="direct:c"/>
+            <to uri="mock:c"/>
+        </route>
+
+    </camelContext>
+
+</beans>

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

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

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

Added: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/streamcachingperroute.xml
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/streamcachingperroute.xml?rev=798862&view=auto
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/streamcachingperroute.xml (added)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/streamcachingperroute.xml Wed Jul 29 11:53:48 2009
@@ -0,0 +1,44 @@
+<?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
+    ">
+
+    <camelContext id="camel" streamCache="true" xmlns="http://camel.apache.org/schema/spring">
+
+        <route streamCache="true">
+            <from uri="direct:a"/>
+            <to uri="mock:a"/>
+        </route>
+
+        <route streamCache="false">
+            <from uri="direct:b"/>
+            <to uri="mock:b"/>
+        </route>
+
+        <route>
+            <from uri="direct:c"/>
+            <to uri="mock:c"/>
+        </route>
+
+    </camelContext>
+
+</beans>

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

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

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