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 09:07:05 UTC

svn commit: r798798 - 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/ camel-core/src/main/...

Author: davsclaus
Date: Wed Jul 29 07:07:04 2009
New Revision: 798798

URL: http://svn.apache.org/viewvc?rev=798798&view=rev
Log:
CAMEL-1553: trace can now be set individually on per route, so you can trace specific routes instead of all or nothing.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteManualTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringTracePerRouteTest.java
      - copied, changed from r798534, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringAOPAfterTest.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml
      - copied, changed from r798534, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopbefore.xml
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/DefaultChannel.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/RouteContext.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/seda/SedaUnitOfWorkTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/timer/TimerRouteWithTracerTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/impl/FromMultipleEndpointTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/MulticastUnitOfWorkTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/aggregator/AggregatorTimerAndTracerTest.java
    camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -482,4 +482,12 @@
      * @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();
+
 }

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -24,7 +24,6 @@
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
-
 import javax.naming.Context;
 
 import org.apache.camel.CamelContext;
@@ -52,7 +51,6 @@
 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.TraceFormatter;
 import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.spi.ClassResolver;
 import org.apache.camel.spi.ComponentResolver;
@@ -73,14 +71,13 @@
 import org.apache.camel.util.LRUCache;
 import org.apache.camel.util.ObjectHelper;
 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;
 
-import static org.apache.camel.util.ServiceHelper.startServices;
-import static org.apache.camel.util.ServiceHelper.stopServices;
-
 /**
  * Represents the context used to configure routes and the policies to use.
  *
@@ -109,7 +106,7 @@
     private LifecycleStrategy lifecycleStrategy;
     private final List<RouteDefinition> routeDefinitions = new ArrayList<RouteDefinition>();
     private List<InterceptStrategy> interceptStrategies = new ArrayList<InterceptStrategy>();
-    private Boolean trace;
+    private Boolean trace = Boolean.FALSE;
     private Boolean streamCache = Boolean.FALSE;
     private Boolean handleFault = Boolean.FALSE;
     private Long delay;
@@ -739,12 +736,11 @@
 
     public void addInterceptStrategy(InterceptStrategy interceptStrategy) {
         getInterceptStrategies().add(interceptStrategy);
+        if (interceptStrategy instanceof Tracer) {
+            setTracing(true);
+        }
     }
 
-    /**
-     * Returns true if tracing has been enabled or disabled via the {@link #setTrace(Boolean)} method
-     * or it has not been specified then default to the <b>camel.streamCache</b> system property
-     */
     public boolean isStreamCacheEnabled() {
         final Boolean value = getStreamCaching();
         if (value != null) {
@@ -763,19 +759,11 @@
     }
 
     public void setTracing(Boolean tracing) {
-        // look if alredy enabled
-        Tracer tracer = Tracer.getTracer(this);
-        if (tracing && tracer == null) {
-            // not already enabled
-            addInterceptStrategy(new Tracer());
-        } else if (tracer != null) {
-            // disable existing tracer
-            for (InterceptStrategy strategy : interceptStrategies) {
-                if (strategy instanceof Tracer) {
-                    interceptStrategies.remove(strategy);
-                }
-            }
-        }
+        this.trace = tracing;
+    }
+
+    public boolean isTracing() {
+        return trace;
     }
 
     /**
@@ -802,7 +790,7 @@
      * Returns true if tracing has been enabled
      */
     public boolean isTraceEnabled() {
-        final Boolean value = getTrace();
+        final Boolean value = isTracing();
         if (value != null) {
             return value;
         } else {
@@ -810,14 +798,6 @@
         }
     }
 
-    public Boolean getTrace() {
-        return trace;
-    }
-
-    public void setTrace(Boolean trace) {
-        this.trace = trace;
-    }
-
     /**
      * Returns the delay in millis if delaying has been enabled. Returns 0 if not enabled.
      */
@@ -907,13 +887,8 @@
 
         if (isTraceEnabled()) {
             // only add a new tracer if not already configured
-            if (Tracer.getTracer(this) == null) {
-                Tracer tracer = new Tracer();
-                // lets see if we have a formatter if so use it
-                TraceFormatter formatter = this.getRegistry().lookup("traceFormatter", TraceFormatter.class);
-                if (formatter != null) {
-                    tracer.setFormatter(formatter);
-                }
+            if (Tracer.getTracer(this.getInterceptStrategies()) == null) {
+                Tracer tracer = Tracer.createTracer(this);
                 LOG.debug("Tracing is enabled");
                 addInterceptStrategy(tracer);
             }

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -49,6 +49,7 @@
     private CamelContext camelContext;
     private List<InterceptStrategy> interceptStrategies = new ArrayList<InterceptStrategy>();
     private boolean routeAdded;
+    private Boolean trace;
 
     public DefaultRouteContext(RouteDefinition route, FromDefinition from, Collection<Route> routes) {
         this.route = route;
@@ -169,6 +170,19 @@
         this.routeAdded = routeAdded;
     }
 
+    public void setTracing(Boolean tracing) {
+        this.trace = tracing;
+    }
+
+    public boolean isTracing() {
+        if (trace != null) {
+            return trace;
+        } else {
+            // fallback to let the camel context decide whether tracing is enabled
+            return getCamelContext().isTracing();
+        }
+    }
+
     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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -50,6 +50,7 @@
 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.Tracer;
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.IdempotentRepository;
@@ -159,10 +160,10 @@
         channel.setNextProcessor(processor);
 
         // add interceptor strategies to the channel
-        channel.addInterceptStrategies(routeContext.getCamelContext().getInterceptStrategies());
-        channel.addInterceptStrategies(routeContext.getInterceptStrategies());
-        channel.addInterceptStrategies(this.getInterceptStrategies());
-
+        addInterceptStrategies(routeContext, channel, routeContext.getCamelContext().getInterceptStrategies());
+        addInterceptStrategies(routeContext, channel, routeContext.getInterceptStrategies());
+        addInterceptStrategies(routeContext, channel, this.getInterceptStrategies());
+        
         // init the channel
         channel.initChannel(this, routeContext);
 
@@ -180,6 +181,23 @@
     }
 
     /**
+     * Adds the given list of interceptors to the channel.
+     *
+     * @param routeContext  the route context
+     * @param channel       the channel to add strategies
+     * @param strategies    list of strategies to add.
+     */
+    protected void addInterceptStrategies(RouteContext routeContext, Channel channel, List<InterceptStrategy> strategies) {
+        for (InterceptStrategy strategy : strategies) {
+            if (!routeContext.isTracing() && strategy instanceof Tracer) {
+                // trace is disabled so we should not add it
+                continue;
+            }
+            channel.addInterceptStrategy(strategy);
+        }
+    }
+
+    /**
      * Creates a new instance of some kind of composite processor which defaults
      * to using a {@link Pipeline} but derived classes could change the behaviour
      */

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -36,6 +36,7 @@
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.impl.DefaultRouteContext;
 import org.apache.camel.processor.interceptor.StreamCaching;
+import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.CamelContextHelper;
 
@@ -53,6 +54,7 @@
     private CamelContext camelContext;
     private String group;
     private Boolean streamCache;
+    private Boolean trace;
 
     public RouteDefinition() {
     }
@@ -195,7 +197,7 @@
     }
 
     /**
-     * Disable stream caching for this Route.
+     * Disable stream caching for this route.
      */
     public RouteDefinition noStreamCaching() {
         setStreamCache(Boolean.FALSE);
@@ -204,7 +206,7 @@
     }
 
     /**
-     * Enable stream caching for this Route.
+     * Enable stream caching for this route.
      */
     public RouteDefinition streamCaching() {
         setStreamCache(Boolean.TRUE);
@@ -217,6 +219,22 @@
         return this;
     }
 
+    /**
+     * Disable tracing for this route.
+     */
+    public RouteDefinition noTracing() {
+        setTrace(false);
+        return this;
+    }
+
+    /**
+     * Enable tracing for this route.
+     */
+    public RouteDefinition tracing() {
+        setTrace(true);
+        return this;
+    }
+
     // Properties
     // -----------------------------------------------------------------------
 
@@ -277,10 +295,32 @@
         this.streamCache = streamCache;
     }
 
+    public Boolean isTrace() {
+        return trace;
+    }
+
+    @XmlAttribute
+    public void setTrace(Boolean trace) {
+        this.trace = trace;
+    }
+
     // Implementation methods
     // -------------------------------------------------------------------------
     protected RouteContext addRoutes(Collection<Route> routes, FromDefinition fromType) throws Exception {
         RouteContext routeContext = new DefaultRouteContext(this, fromType, routes);
+
+        // configure tracing
+        if (trace != null) {
+            routeContext.setTracing(isTrace());
+            if (isTrace()) {
+                // only add a new tracer if not already a global configured on camel context
+                if (Tracer.getTracer(camelContext.getInterceptStrategies()) == null) {
+                    Tracer tracer = Tracer.createTracer(camelContext);
+                    addInterceptStrategy(tracer);
+                }
+            }
+        }
+
         // 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/DefaultChannel.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultChannel.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultChannel.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/DefaultChannel.java Wed Jul 29 07:07:04 2009
@@ -71,7 +71,7 @@
     public Processor getOutput() {
         // the errorHandler is already decorated with interceptors
         // so it cointain the entire chain of processors, so we can safely use it directly as output
-        // if no error handler provided we can use the output direcly
+        // if no error handler provided we use the output
         return errorHandler != null ? errorHandler : output;
     }
 

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -18,11 +18,11 @@
 
 import java.util.List;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
+import org.apache.camel.CamelContext;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.InterceptStrategy;
 
@@ -47,13 +47,27 @@
     private boolean useJpa;
 
     /**
-     * A helper method to return the Tracer instance for a given {@link CamelContext} if one is enabled
+     * Creates a new tracer.
+     *
+     * @param context Camel context
+     * @return a new tracer
+     */
+    public static Tracer createTracer(CamelContext context) {
+        Tracer tracer = new Tracer();
+        // lets see if we have a formatter if so use it
+        TraceFormatter formatter = context.getRegistry().lookup("traceFormatter", TraceFormatter.class);
+        if (formatter != null) {
+            tracer.setFormatter(formatter);
+        }
+        return tracer;
+    }
+
+    /**
+     * A helper method to return the Tracer instance if one is enabled
      *
-     * @param context the camel context the tracer is connected to
      * @return the tracer or null if none can be found
      */
-    public static Tracer getTracer(CamelContext context) {
-        List<InterceptStrategy> list = context.getInterceptStrategies();
+    public static Tracer getTracer(List<InterceptStrategy> list) {
         for (InterceptStrategy interceptStrategy : list) {
             if (interceptStrategy instanceof Tracer) {
                 return (Tracer)interceptStrategy;

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -163,4 +163,19 @@
      * @return the found object
      */
     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();
+
 }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/seda/SedaUnitOfWorkTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/seda/SedaUnitOfWorkTest.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/seda/SedaUnitOfWorkTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/seda/SedaUnitOfWorkTest.java Wed Jul 29 07:07:04 2009
@@ -55,7 +55,7 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                context.addInterceptStrategy(new Tracer());
+                context.setTracing(true);
 
                 from("direct:start")
                         .process(new MyUOWProcessor("A"))

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/timer/TimerRouteWithTracerTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/timer/TimerRouteWithTracerTest.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/timer/TimerRouteWithTracerTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/timer/TimerRouteWithTracerTest.java Wed Jul 29 07:07:04 2009
@@ -29,7 +29,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() {
-                getContext().addInterceptStrategy(new Tracer());
+                getContext().setTracing(true);
                 from("timer://foo?fixedRate=true&delay=0&period=500").to("bean:myBean", "mock:result");
             }
         };

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/impl/FromMultipleEndpointTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/FromMultipleEndpointTest.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/impl/FromMultipleEndpointTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/impl/FromMultipleEndpointTest.java Wed Jul 29 07:07:04 2009
@@ -54,7 +54,7 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                getContext().addInterceptStrategy(new Tracer());
+                getContext().setTracing(true);
 
                 from("direct:foo", "seda:bar", "timer://baz?delay=500&period=1000").to("mock:results");
             }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/MulticastUnitOfWorkTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/MulticastUnitOfWorkTest.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/MulticastUnitOfWorkTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/MulticastUnitOfWorkTest.java Wed Jul 29 07:07:04 2009
@@ -53,7 +53,7 @@
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                context.addInterceptStrategy(new Tracer());
+                context.setTracing(true);
 
                 from("direct:start")
                         .process(new MyUOWProcessor("A"))

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteManualTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteManualTest.java?rev=798798&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteManualTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteManualTest.java Wed Jul 29 07:07:04 2009
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class TracePerRouteManualTest extends ContextTestSupport {
+
+    public void testTracingPerRoute() throws Exception {
+        getMockEndpoint("mock:a").expectedMessageCount(1);
+        getMockEndpoint("mock:b").expectedMessageCount(1);
+        getMockEndpoint("mock:c").expectedMessageCount(1);
+
+        template.sendBody("direct:a", "Hello World");
+        template.sendBody("direct:b", "Bye World");
+        template.sendBody("direct:c", "Gooday World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:a").tracing().streamCaching().to("mock:a");
+
+                from("direct:b").noTracing().to("mock:b");
+
+                from("direct:c").tracing().to("mock:c");
+            }
+        };
+    }
+}
\ No newline at end of file

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

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

Added: 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/TracePerRouteTest.java?rev=798798&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/TracePerRouteTest.java Wed Jul 29 07:07:04 2009
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.processor.interceptor.Tracer;
+
+/**
+ * @version $Revision$
+ */
+public class TracePerRouteTest 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");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                Tracer tracer = new Tracer();
+                tracer.setDestinationUri("mock:traced");
+                tracer.setLogName("foo");
+                context.addInterceptStrategy(tracer);
+
+                from("direct:a").to("mock:a");
+
+                from("direct:b").noTracing().to("mock:b");
+
+                from("direct:c").tracing().to("mock:c");
+            }
+        };
+    }
+}

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

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

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/aggregator/AggregatorTimerAndTracerTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/aggregator/AggregatorTimerAndTracerTest.java?rev=798798&r1=798797&r2=798798&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/aggregator/AggregatorTimerAndTracerTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/aggregator/AggregatorTimerAndTracerTest.java Wed Jul 29 07:07:04 2009
@@ -37,7 +37,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                getContext().addInterceptStrategy(new Tracer());
+                getContext().setTracing(true);
 
                 from("timer://kickoff?period=9999910000").
                     setHeader("id").constant("foo").setBody().constant("a b c").

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=798798&r1=798797&r2=798798&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 07:07:04 2009
@@ -714,7 +714,7 @@
             ctx.setStreamCaching(streamCache);
         }
         if (trace != null) {
-            ctx.setTrace(trace);
+            ctx.setTracing(trace);
         }
         if (delay != null) {
             ctx.setDelay(delay);

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

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml (from r798534, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopbefore.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopbefore.xml&r1=798534&r2=798798&rev=798798&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/aopbefore.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/traceperroute.xml Wed Jul 29 07:07:04 2009
@@ -22,16 +22,24 @@
        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">
+    <bean id="myTracer" class="org.apache.camel.processor.interceptor.Tracer">
+        <property name="destinationUri" value="mock:traced"/>
+        <property name="logName" value="foo"/>
+    </bean>
+
+    <camelContext id="camel" trace="true" xmlns="http://camel.apache.org/schema/spring">
         <route>
-            <from uri="direct:start"/>
-            <aop beforeUri="mock:before">
-                <transform><constant>Bye World</constant></transform>
-                <to uri="mock:result"/>
-            </aop>
+            <from uri="direct:a"/>
+            <to uri="mock:a"/>
+        </route>
+        <route trace="false">
+            <from uri="direct:b"/>
+            <to uri="mock:b"/>
+        </route>
+        <route trace="true">
+            <from uri="direct:c"/>
+            <to uri="mock:c"/>
         </route>
     </camelContext>
-    <!-- END SNIPPET: e1 -->
 
 </beans>