You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2009/05/02 09:54:38 UTC

svn commit: r770908 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/builder/ camel-core/src/main/java/org/apache/camel/management/ camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/processor/ camel-...

Author: davsclaus
Date: Sat May  2 07:54:36 2009
New Revision: 770908

URL: http://svn.apache.org/viewvc?rev=770908&view=rev
Log:
CAMEL-1569: Added intercept DSL that works like a real interceptor applied at each and every processing step.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java
      - copied, changed from r770770, camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteStopTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.java   (contents, props changed)
      - copied, changed from r770770, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptFromUriSimpleLogTest.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml
      - copied, changed from r770754, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptFromTest.xml
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationInterceptStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.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/RoutesDefinition.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/Debugger.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Delayer.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/HandleFault.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/StreamCaching.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/InterceptStrategy.java
    camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
    camel/trunk/camel-core/src/test/java/org/apache/camel/issues/TwoTimerWithJMXIssue.java
    camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java
    camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java Sat May  2 07:54:36 2009
@@ -22,11 +22,10 @@
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
-import org.apache.camel.Predicate;
 import org.apache.camel.Route;
 import org.apache.camel.Routes;
 import org.apache.camel.impl.DefaultCamelContext;
-import org.apache.camel.model.ChoiceDefinition;
+import org.apache.camel.model.InterceptDefinition;
 import org.apache.camel.model.InterceptFromDefinition;
 import org.apache.camel.model.InterceptSendToEndpointDefinition;
 import org.apache.camel.model.OnExceptionDefinition;
@@ -151,6 +150,17 @@
      *
      * @return the builder
      */
+    public InterceptDefinition intercept() {
+        routeCollection.setCamelContext(getContext());
+        return routeCollection.intercept();
+    }
+
+    /**
+     * Adds a route for an interceptor; use the {@link org.apache.camel.model.ProcessorDefinition#proceed()} method
+     * to continue processing the underlying route being intercepted.
+     *
+     * @return the builder
+     */
     public InterceptFromDefinition interceptFrom() {
         routeCollection.setCamelContext(getContext());
         return routeCollection.interceptFrom();

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationInterceptStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationInterceptStrategy.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationInterceptStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationInterceptStrategy.java Sat May  2 07:54:36 2009
@@ -40,7 +40,7 @@
         this.registeredCounters = registeredCounters;
     }
 
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         // dont double wrap it
         if (target instanceof InstrumentationProcessor) {
             return target;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java Sat May  2 07:54:36 2009
@@ -258,7 +258,7 @@
 
         // add an intercept strategy that counts when the route sends to any of its outputs
         out.addInterceptStrategy(new InterceptStrategy() {
-            public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+            public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
                 if (registeredRoutes.containsKey(endpoint)) {
                     // do not double wrap
                     return target;

Copied: camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java (from r770770, camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java?p2=camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java&r1=770770&r2=770908&rev=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java Sat May  2 07:54:36 2009
@@ -16,89 +16,77 @@
  */
 package org.apache.camel.model;
 
-import java.util.ArrayList;
 import java.util.List;
+import java.util.ArrayList;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
-import javax.xml.bind.annotation.XmlAttribute;
 
+import org.apache.camel.Exchange;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
-import org.apache.camel.builder.PredicateBuilder;
+import org.apache.camel.processor.Pipeline;
+import org.apache.camel.spi.InterceptStrategy;
 import org.apache.camel.spi.RouteContext;
 
 /**
- * Represents an XML <interceptFrom/> element
+ * Represents an XML <intercept/> element
  *
  * @version $Revision$
  */
-@XmlRootElement(name = "interceptFrom")
+@XmlRootElement(name = "intercept")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class InterceptFromDefinition extends OutputDefinition<ProcessorDefinition> {
+public class InterceptDefinition extends OutputDefinition<ProcessorDefinition> {
 
-    // TODO: we need a new interceptDefinition to uses the InterceptStrategy so its applied for each route step
-    // and a more intelligent Channel so we can stop or proceed on-the-fly
+    // TODO: support stop later (its a bit hard as it needs to break entire processing of route)
+    // TODO: add support for when predicate
 
-    // TODO: Support lookup endpoint by ref (requires a bit more work)
-    // TODO: Support wildcards for endpoints so you can match by scheme, eg jms:*
-
-    @XmlAttribute(required = false)
-    private String uri;
-    @XmlTransient
-    protected ProceedDefinition proceed = new ProceedDefinition();
     @XmlTransient
-    protected Boolean stopIntercept = Boolean.FALSE;
-    @XmlTransient
-    protected Boolean usePredicate = Boolean.FALSE;
-
-    public InterceptFromDefinition() {
-    }
+    protected Processor output;
 
-    public InterceptFromDefinition(String uri) {
-        this.uri = uri;
+    public InterceptDefinition() {
     }
 
     @Override
     public String toString() {
-        return "InterceptFrom[" + getOutputs() + "]";
+        return "Intercept[" + getOutputs() + "]";
     }
 
     @Override
     public String getShortName() {
-        return "interceptFrom";
+        return "intercept";
     }
 
     @Override
     public String getLabel() {
-        return "interceptFrom";
+        return "intercept";
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return createOutputsProcessor(routeContext);
-    }
-
-    /**
-     * Applies this interceptor only if the given predicate is true
-     *
-     * @param predicate  the predicate
-     * @return the builder
-     */
-    public ChoiceDefinition when(Predicate predicate) {
-        usePredicate = Boolean.TRUE;
-        ChoiceDefinition choice = choice().when(PredicateBuilder.not(predicate));
-        choice.addOutput(proceed);
-        return choice.otherwise();
-    }
-
-    public ProceedDefinition getProceed() {
-        return proceed;
-    }
+    public Processor createProcessor(final RouteContext routeContext) throws Exception {
+        // create the output processor
+        output = createOutputsProcessor(routeContext);
+
+        // add the output as a intercept strategy to the route context so its invoked on each processing step
+        routeContext.getInterceptStrategies().add(new InterceptStrategy() {
+            public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
+                if (nextTarget != null) {
+                    // wrap in a pipeline so we continue routing to the next
+                    List<Processor> list = new ArrayList<Processor>(2);
+                    list.add(output);
+                    list.add(nextTarget);
+                    return new Pipeline(list);
+                } else {
+                    return output;
+                }
+            }
+        });
 
-    public void stopIntercept() {
-        setStopIntercept(Boolean.TRUE);
+        // remove me from the route so I am not invoked in a regular route path
+        routeContext.getRoute().getOutputs().remove(this);
+        // and return no processor to invoke next from me
+        return null;
     }
 
     /**
@@ -109,123 +97,8 @@
      * with or without proceed/stop set as well.
      */
     public void afterPropertiesSet() {
-        List<ProcessorDefinition> list = new ArrayList<ProcessorDefinition>();
-        for (ProcessorDefinition out : outputs) {
-            if (out instanceof WhenDefinition) {
-                // JAXB does not invoke the when() fluent builder so we need to wrap the when in
-                // a choice with the proceed as the when for the Java DSL does
-                WhenDefinition when = (WhenDefinition) out;
-                usePredicate = Boolean.TRUE;
-                ChoiceDefinition choice = new ChoiceDefinition();
-                choice.when(PredicateBuilder.not(when.getExpression()));
-                choice.addOutput(proceed);
-                list.add(choice);
-
-                ChoiceDefinition otherwise = choice.otherwise();
-                // add the children to the otherwise
-                for (ProcessorDefinition child : when.getOutputs()) {
-                    if (child instanceof StopDefinition) {
-                        // notify we should stop
-                        stopIntercept();
-                    } else {
-                        otherwise.addOutput(child);
-                    }
-                }
-            } else if (out instanceof StopDefinition) {
-                // notify we shuld stop
-                stopIntercept();
-            } else {
-                list.add(out);
-            }
-        }
-
-        // replace old output with this redone output list
-        outputs.clear();
-        for (ProcessorDefinition out : list) {
-            addOutput(out);
-        }
-    }
-
-    // TODO: reduce complexity of this code
-
-    public InterceptFromDefinition createProxy() {
-        InterceptFromDefinition answer = new InterceptFromDefinition();
-        answer.getOutputs().addAll(this.getOutputs());
-        
-        answer.setStopIntercept(getStopIntercept());
-
-        // hack: now we need to replace the proceed of the proxy with its own
-        // a bit ugly, operating based on the assumption that the proceed is
-        // in its outputs (if proceed() was called) and/or in the
-        // outputs of the otherwise or last when clause for the predicated version.
-        if (answer.getOutputs().size() > 0) {
-            // this is for the predicate version or if a choice() is present
-            ChoiceDefinition choice = null;
-            for (ProcessorDefinition processor : answer.getOutputs()) {
-                if (processor instanceof ChoiceDefinition) {
-                    // special cases for predicates (choices)
-                    choice = (ChoiceDefinition) processor;
-
-                    // for the predicated version we add the proceed() to otherwise()
-                    // before knowing if stop() will follow, so let's make a small adjustment
-                    if (usePredicate && getStopIntercept()) {
-                        WhenDefinition when = choice.getWhenClauses().get(0);
-                        when.getOutputs().remove(this.getProceed());
-                    }
-
-                    // add proceed to the when clause
-                    addProceedProxy(this.getProceed(), answer.getProceed(),
-                        choice.getWhenClauses().get(choice.getWhenClauses().size() - 1), usePredicate && !getStopIntercept());
-
-                    // force adding a proceed at the end (otherwise) if its not a stop type
-                    addProceedProxy(this.getProceed(), answer.getProceed(), choice.getOtherwise(), !getStopIntercept());
-
-                    if (getStopIntercept()) {
-                        // must add proceed to when clause if stop is explictiy declared, otherwise when the
-                        // predicate test fails then there is no proceed
-                        // See example: InterceptFromSimpleRouteTest (City Paris is never proceeded)
-                        addProceedProxy(this.getProceed(), answer.getProceed(),
-                            choice.getWhenClauses().get(choice.getWhenClauses().size() - 1), usePredicate);
-                    }
-
-                    break;
-                }
-            }
-            if (choice == null) {
-                // force adding a proceed at the end if its not a stop type
-                addProceedProxy(this.getProceed(), answer.getProceed(), answer, !getStopIntercept());
-            }
-        }
-
-        return answer;
+        // TODO: is needed when we add support for when predicate
     }
 
-    private void addProceedProxy(ProceedDefinition orig, ProceedDefinition proxy, ProcessorDefinition<?> processor, boolean force) {
-        int index = processor.getOutputs().indexOf(orig);
-        if (index >= 0) {
-            processor.addOutput(proxy);
-            // replace original proceed with proxy
-            List<ProcessorDefinition> outs = processor.getOutputs();
-            outs.remove(proxy);
-            outs.set(index, proxy);
-        } else if (force) {
-            processor.addOutput(proxy);
-        }
-    }
-
-    public void setStopIntercept(Boolean stop) {
-        this.stopIntercept = stop;
-    }
-
-    public Boolean getStopIntercept() {
-        return stopIntercept;
-    }
-
-    public String getUri() {
-        return uri;
-    }
 
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-}
+}
\ No newline at end of file

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=770908&r1=770907&r2=770908&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 Sat May  2 07:54:36 2009
@@ -109,6 +109,11 @@
 
     public void addRoutes(RouteContext routeContext, Collection<Route> routes) throws Exception {
         Processor processor = makeProcessor(routeContext);
+        if (processor == null) {
+            // no processor to add
+            return;
+        }
+
         if (!routeContext.isRouteAdded()) {
             boolean endpointInterceptor = false;
 
@@ -219,6 +224,10 @@
      */
     protected Processor makeProcessor(RouteContext routeContext) throws Exception {
         Processor processor = createProcessor(routeContext);
+        if (processor == null) {
+            // no processor to make
+            return null;
+        }
         return wrapProcessor(routeContext, processor);
     }
 
@@ -1342,7 +1351,6 @@
             if (proceed == null) {
                 throw new IllegalArgumentException("Cannot use proceed() without being within an intercept() block");
             }
-
         }
 
         addOutput(proceed);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutesDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutesDefinition.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutesDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutesDefinition.java Sat May  2 07:54:36 2009
@@ -40,6 +40,8 @@
     @XmlElementRef
     private List<RouteDefinition> routes = new ArrayList<RouteDefinition>();
     @XmlTransient
+    private List<InterceptDefinition> intercepts = new ArrayList<InterceptDefinition>();
+    @XmlTransient
     private List<InterceptFromDefinition> interceptFroms = new ArrayList<InterceptFromDefinition>();
     @XmlTransient
     private List<InterceptSendToEndpointDefinition> interceptSendTos = new ArrayList<InterceptSendToEndpointDefinition>();
@@ -86,6 +88,14 @@
         this.interceptSendTos = interceptSendTos;
     }
 
+    public List<InterceptDefinition> getIntercepts() {
+        return intercepts;
+    }
+
+    public void setIntercepts(List<InterceptDefinition> intercepts) {
+        this.intercepts = intercepts;
+    }
+
     public List<OnExceptionDefinition> getOnExceptions() {
         return onExceptions;
     }
@@ -181,6 +191,11 @@
         // lets configure the route
         route.setCamelContext(getCamelContext());
 
+        // configure intercepts
+        for (InterceptDefinition intercept : getIntercepts()) {
+            route.addOutput(intercept);
+        }
+
         // configure intercept from
         for (InterceptFromDefinition intercept : getInterceptFroms()) {
 
@@ -206,8 +221,7 @@
         }
 
         // configure intercept send to endpoint
-        List<InterceptSendToEndpointDefinition> sendTos = getInterceptSendTos();
-        for (InterceptSendToEndpointDefinition sendTo : sendTos) {
+        for (InterceptSendToEndpointDefinition sendTo : getInterceptSendTos()) {
             // init interceptor by letting it proxy the real endpoint
             sendTo.proxyEndpoint(getCamelContext());
             route.addOutput(sendTo);
@@ -221,6 +235,18 @@
     }
 
     /**
+     * Creates and adds an interceptor that is triggered on every step in the route
+     * processing.
+     *
+     * @return the interceptor builder to configure
+     */
+    public InterceptDefinition intercept() {
+        InterceptDefinition answer = new InterceptDefinition();
+        getIntercepts().add(answer);
+        return answer;
+    }
+
+    /**
      * Creates and adds an interceptor that is triggered when an exchange
      * is received as input to any routes (eg from all the <tt>from</tt>)
      *

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=770908&r1=770907&r2=770908&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 Sat May  2 07:54:36 2009
@@ -130,7 +130,7 @@
         // wrap the output with the interceptors
         Processor target = nextProcessor;
         for (InterceptStrategy strategy : interceptors) {
-            target = strategy.wrapProcessorInInterceptors(outputDefinition, target);
+            target = strategy.wrapProcessorInInterceptors(outputDefinition, target, nextProcessor);
         }
 
         // sets the delegate to our wrapped output

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debugger.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debugger.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debugger.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debugger.java Sat May  2 07:54:36 2009
@@ -115,7 +115,7 @@
     }
 
 
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         String id = processorDefinition.idOrCreate();
         if (logExchanges) {
             TraceInterceptor traceInterceptor = new TraceInterceptor(processorDefinition, target, tracer);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Delayer.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Delayer.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Delayer.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Delayer.java Sat May  2 07:54:36 2009
@@ -54,7 +54,7 @@
         return null;
     }
 
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         DelayInterceptor delayer = new DelayInterceptor(processorDefinition, target, this);
         return delayer;
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/HandleFault.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/HandleFault.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/HandleFault.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/HandleFault.java Sat May  2 07:54:36 2009
@@ -29,7 +29,7 @@
 public final class HandleFault implements InterceptStrategy {
 
     @SuppressWarnings("unchecked")
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         return new HandleFaultInterceptor(target);
     }
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/StreamCaching.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/StreamCaching.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/StreamCaching.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/StreamCaching.java Sat May  2 07:54:36 2009
@@ -29,7 +29,7 @@
 public final class StreamCaching implements InterceptStrategy {
     
     @SuppressWarnings("unchecked")
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         return new StreamCachingInterceptor(target);
     }
     

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=770908&r1=770907&r2=770908&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 Sat May  2 07:54:36 2009
@@ -61,7 +61,7 @@
         return null;
     }
 
-    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception {
+    public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget) throws Exception {
         // Force the creation of an id, otherwise the id is not available when the trace formatter is
         // outputting trace information
         String id = processorDefinition.idOrCreate();

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/InterceptStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/InterceptStrategy.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/InterceptStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/InterceptStrategy.java Sat May  2 07:54:36 2009
@@ -36,8 +36,10 @@
      *
      * @param processorDefinition the object that invokes this method
      * @param target        the processor to be wrapped
+     * @param nextTarget    the next processor to be routed to
      * @return processor wrapped with an interceptor or not wrapped
      * @throws Exception can be thrown
      */
-    Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target) throws Exception;
+    Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition,
+                                          Processor target, Processor nextTarget) throws Exception;
 }

Modified: camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index (original)
+++ camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index Sat May  2 07:54:36 2009
@@ -29,6 +29,7 @@
 IdempotentConsumerDefinition
 InOnlyDefinition
 InOutDefinition
+InterceptDefinition
 InterceptFromDefinition
 InterceptSendToEndpointDefinition
 LoadBalanceDefinition

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/issues/TwoTimerWithJMXIssue.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/issues/TwoTimerWithJMXIssue.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/issues/TwoTimerWithJMXIssue.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/issues/TwoTimerWithJMXIssue.java Sat May  2 07:54:36 2009
@@ -53,7 +53,7 @@
     }
 
     private class MyTracer implements InterceptStrategy {
-        public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target)
+        public Processor wrapProcessorInInterceptors(ProcessorDefinition processorDefinition, Processor target, Processor nextTarget)
             throws Exception {
             assertNotNull(target);
             counter++;

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteStopTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteStopTest.java?rev=770908&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteStopTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteStopTest.java Sat May  2 07:54:36 2009
@@ -0,0 +1,53 @@
+/**
+ * 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.intercept;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class InterceptSimpleRouteStopTest extends ContextTestSupport {
+
+    public void testInterceptWithStop() throws Exception {
+        // TODO: stop is not yet supported
+        //getMockEndpoint("mock:foo").expectedMessageCount(0);
+        //getMockEndpoint("mock:bar").expectedMessageCount(0);
+        //getMockEndpoint("mock:result").expectedMessageCount(0);
+
+        //getMockEndpoint("mock:intercepted").expectedMessageCount(1);
+
+        //template.sendBody("direct:start", "Hello World");
+
+        //assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+//                intercept().to("mock:intercepted").stop();
+                intercept().to("mock:intercepted");
+
+                from("direct:start")
+                    .to("mock:foo", "mock:bar", "mock:result");
+            }
+        };
+    }
+}
\ No newline at end of file

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteTest.java?rev=770908&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/intercept/InterceptSimpleRouteTest.java Sat May  2 07:54:36 2009
@@ -0,0 +1,51 @@
+/**
+ * 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.intercept;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * @version $Revision$
+ */
+public class InterceptSimpleRouteTest extends ContextTestSupport {
+
+    public void testIntercept() throws Exception {
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        getMockEndpoint("mock:bar").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        getMockEndpoint("mock:intercepted").expectedMessageCount(3);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                intercept().to("mock:intercepted");
+
+                from("direct:start")
+                    .to("mock:foo", "mock:bar", "mock:result");
+            }
+        };
+    }
+}

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

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

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=770908&r1=770907&r2=770908&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 Sat May  2 07:54:36 2009
@@ -18,7 +18,6 @@
 
 import java.util.ArrayList;
 import java.util.List;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -29,12 +28,15 @@
 
 import org.apache.camel.CamelException;
 import org.apache.camel.Routes;
+import org.apache.camel.Processor;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.DefaultLifecycleStrategy;
 import org.apache.camel.management.DefaultInstrumentationAgent;
 import org.apache.camel.management.InstrumentationLifecycleStrategy;
+import org.apache.camel.model.FromDefinition;
 import org.apache.camel.model.IdentifiedType;
+import org.apache.camel.model.InterceptDefinition;
 import org.apache.camel.model.InterceptFromDefinition;
 import org.apache.camel.model.InterceptSendToEndpointDefinition;
 import org.apache.camel.model.OnExceptionDefinition;
@@ -45,7 +47,6 @@
 import org.apache.camel.model.RouteContainer;
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.model.TransactedDefinition;
-import org.apache.camel.model.FromDefinition;
 import org.apache.camel.model.config.PropertiesDefinition;
 import org.apache.camel.model.dataformat.DataFormatsDefinition;
 import org.apache.camel.processor.interceptor.Debugger;
@@ -58,6 +59,7 @@
 import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.spi.PackageScanClassResolver;
 import org.apache.camel.spi.Registry;
+import org.apache.camel.spi.InterceptStrategy;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.ProcessorDefinitionHelper;
 import org.apache.commons.logging.Log;
@@ -71,10 +73,8 @@
 import org.springframework.context.ApplicationEvent;
 import org.springframework.context.ApplicationListener;
 import org.springframework.context.event.ContextRefreshedEvent;
-
 import static org.apache.camel.util.ObjectHelper.wrapRuntimeCamelException;
 
-
 /**
  * A Spring {@link FactoryBean} to create and initialize a
  * {@link SpringCamelContext} and install routes either explicitly configured in
@@ -121,6 +121,8 @@
     private DataFormatsDefinition dataFormats;
     @XmlElement(name = "onException", required = false)
     private List<OnExceptionDefinition> onExceptions = new ArrayList<OnExceptionDefinition>();
+    @XmlElement(name = "intercept", required = false)
+    private List<InterceptDefinition> intercepts = new ArrayList<InterceptDefinition>();
     @XmlElement(name = "interceptFrom", required = false)
     private List<InterceptFromDefinition> interceptFroms = new ArrayList<InterceptFromDefinition>();
     @XmlElement(name = "interceptSendToEndpoint", required = false)
@@ -288,8 +290,17 @@
     }
 
     private void initInterceptors(RouteDefinition route) {
+
+        // configure intercept
+        for (InterceptDefinition intercept : getIntercepts()) {
+            intercept.afterPropertiesSet();
+            // add as first output so intercept is handled before the acutal route and that gives
+            // us the needed head start to init and be able to intercept all the remaining processing steps
+            route.getOutputs().add(0, intercept);
+        }
+
         // configure intercept from
-        for (InterceptFromDefinition intercept : interceptFroms) {
+        for (InterceptFromDefinition intercept : getInterceptFroms()) {
 
             // should we only apply interceptor for a given endpoint uri
             boolean match = true;
@@ -305,6 +316,8 @@
 
             if (match) {
 
+                // TODO: reduce the complex of this code when we overhaul the intercept from
+
                 List<ProcessorDefinition<?>> outputs = new ArrayList<ProcessorDefinition<?>>();
                 List<ProcessorDefinition<?>> exceptionHandlers = new ArrayList<ProcessorDefinition<?>>();
 
@@ -338,7 +351,7 @@
         }
 
         // configure intercept send to endpoint
-        for (InterceptSendToEndpointDefinition intercept : interceptSendToEndpoints) {
+        for (InterceptSendToEndpointDefinition intercept : getInterceptSendToEndpoints()) {
             // special intercept for intercepting sending to an endpoint
             // init interceptor by letting it proxy the real endpoint
 
@@ -467,6 +480,14 @@
         this.routes = routes;
     }
 
+    public List<InterceptDefinition> getIntercepts() {
+        return intercepts;
+    }
+
+    public void setIntercepts(List<InterceptDefinition> intercepts) {
+        this.intercepts = intercepts;
+    }
+
     public List<InterceptFromDefinition> getInterceptFroms() {
         return interceptFroms;
     }

Modified: camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java?rev=770908&r1=770907&r2=770908&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java (original)
+++ camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java Sat May  2 07:54:36 2009
@@ -202,6 +202,7 @@
                 CamelContextFactoryBean factoryBean = (CamelContextFactoryBean)value;
                 builder.addPropertyValue("id", contextId);
                 builder.addPropertyValue("routes", factoryBean.getRoutes());
+                builder.addPropertyValue("intercepts", factoryBean.getIntercepts());
                 builder.addPropertyValue("interceptFroms", factoryBean.getInterceptFroms());
                 builder.addPropertyValue("interceptSendToEndpoints", factoryBean.getInterceptSendToEndpoints());
                 builder.addPropertyValue("dataFormats", factoryBean.getDataFormats());

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.java (from r770770, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptFromUriSimpleLogTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptFromUriSimpleLogTest.java&r1=770770&r2=770908&rev=770908&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptFromUriSimpleLogTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.java Sat May  2 07:54:36 2009
@@ -19,9 +19,10 @@
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.processor.intercept.InterceptFromUriSimpleLogTest;
+import org.apache.camel.processor.intercept.InterceptSimpleRouteTest;
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
-public class SpringInterceptFromUriSimpleLogTest extends InterceptFromUriSimpleLogTest {
+public class SpringInterceptSimpleRouteTest extends InterceptSimpleRouteTest {
 
     @Override
     public boolean isUseRouteBuilder() {
@@ -29,7 +30,7 @@
     }
 
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringInterceptFromUriSimpleLogTest.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml");
     }
 
 }
\ No newline at end of file

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

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

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml (from r770754, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptFromTest.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptFromTest.xml&r1=770754&r2=770908&rev=770908&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptFromTest.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringInterceptSimpleRouteTest.xml Sat May  2 07:54:36 2009
@@ -24,17 +24,20 @@
 
     <!-- START SNIPPET: example -->
     <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
-        <!-- intercept incoming messages and route them to the mock:middle1 endpoint
-             before we proceed and continue routing from the point of interceptions, that
-             is mock:end will be the next target -->
-        <interceptFrom>
-            <to uri="mock:middle1"/>
-        </interceptFrom>
+        <!-- here we intercept each processing step in the routing and do a detour
+             routing where we route the exhange to the mock:intercepted endpoint.
+             Think the intercept as a AOP around that is weaven around each and
+             every route tag in the route path below -->
+        <intercept>
+            <to uri="mock:intercepted"/>
+        </intercept>
 
         <!-- here we have a very simple route -->
         <route>
             <from uri="direct:start"/>
-            <to uri="mock:end"/>
+            <to uri="mock:foo"/>
+            <to uri="mock:bar"/>
+            <to uri="mock:result"/>
         </route>
     </camelContext>
     <!-- END SNIPPET: example -->