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 2011/02/20 11:22:22 UTC

svn commit: r1072545 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/builder/ main/java/org/apache/camel/model/ test/java/org/apache/camel/processor/interceptor/

Author: davsclaus
Date: Sun Feb 20 10:22:21 2011
New Revision: 1072545

URL: http://svn.apache.org/viewvc?rev=1072545&view=rev
Log:
CAMEL-3689: AdviceWith can now manipulate routes. This allows you for example to replace parts of routes during testing.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTask.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTasks.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksMatchTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksTest.java   (contents, props changed)
      - copied, changed from r1072268, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithRouteIdTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksToStringPatternTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithRouteBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/RouteBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/OutputDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithBuilder.java?rev=1072545&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithBuilder.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithBuilder.java Sun Feb 20 10:22:21 2011
@@ -0,0 +1,97 @@
+/**
+ * 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.builder;
+
+import org.apache.camel.model.PipelineDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+
+/**
+ * A builder when using the <a href="http://camel.apache.org/advicewith.html">advice with</a> feature.
+ */
+public class AdviceWithBuilder {
+
+    private final AdviceWithRouteBuilder builder;
+    private final String id;
+    private final String toString;
+
+    public AdviceWithBuilder(AdviceWithRouteBuilder builder, String id, String toString) {
+        this.builder = builder;
+        this.id = id;
+        this.toString = toString;
+
+        if (id == null && toString == null) {
+            throw new IllegalArgumentException("Either id or toString must be specified");
+        }
+    }
+
+    /**
+     * Replaces the matched node(s) with the following nodes.
+     *
+     * @return the builder to build the nodes.
+     */
+    public ProcessorDefinition replace() {
+        PipelineDefinition answer = new PipelineDefinition();
+        if (id != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.replaceById(builder.getOriginalRoute(), id, answer));
+        } else if (toString != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.replaceByToString(builder.getOriginalRoute(), toString, answer));
+        }
+        return answer;
+    }
+
+    /**
+     * Removes the matched node(s)
+     */
+    public void remove() {
+        if (id != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.removeById(builder.getOriginalRoute(), id));
+        } else if (toString != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.removeByToString(builder.getOriginalRoute(), toString));
+        }
+    }
+
+    /**
+     * Insert the following node(s) <b>before</b> the matched node(s)
+     *
+     * @return the builder to build the nodes.
+     */
+    public ProcessorDefinition before() {
+        PipelineDefinition answer = new PipelineDefinition();
+        if (id != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.beforeById(builder.getOriginalRoute(), id, answer));
+        } else if (toString != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.beforeByToString(builder.getOriginalRoute(), toString, answer));
+        }
+        return answer;
+    }
+
+    /**
+     * Insert the following node(s) <b>after</b> the matched node(s)
+     *
+     * @return the builder to build the nodes.
+     */
+    public ProcessorDefinition after() {
+        PipelineDefinition answer = new PipelineDefinition();
+        if (id != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.afterById(builder.getOriginalRoute(), id, answer));
+        } else if (toString != null) {
+            builder.getAdviceWithTasks().add(AdviceWithTasks.afterByToString(builder.getOriginalRoute(), toString, answer));
+        }
+        return answer;
+    }
+
+}

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithRouteBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithRouteBuilder.java?rev=1072545&r1=1072544&r2=1072545&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithRouteBuilder.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithRouteBuilder.java Sun Feb 20 10:22:21 2011
@@ -16,18 +16,56 @@
  */
 package org.apache.camel.builder;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.camel.impl.InterceptSendToMockEndpointStrategy;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.util.ObjectHelper;
 
 /**
- * A {@link RouteBuilder} which has extended features when using
- * {@link org.apache.camel.model.RouteDefinition#adviceWith(org.apache.camel.CamelContext, RouteBuilder) adviceWith}.
+ * A {@link RouteBuilder} which has extended capabilities when using
+ * the <a href="http://camel.apache.org/advicewith.html">advice with</a> feature.
  *
- * @version 
+ * @see org.apache.camel.model.RouteDefinition#adviceWith(org.apache.camel.CamelContext, RouteBuilder)
  */
 public abstract class AdviceWithRouteBuilder extends RouteBuilder {
 
+    private RouteDefinition originalRoute;
+    private final List<AdviceWithTask> adviceWithTasks = new ArrayList<AdviceWithTask>();
+
+    /**
+     * Sets the original route which we advice.
+     *
+     * @param originalRoute the original route we advice.
+     */
+    public void setOriginalRoute(RouteDefinition originalRoute) {
+        this.originalRoute = originalRoute;
+    }
+
+    /**
+     * Gets the original route we advice.
+     *
+     * @return the original route.
+     */
+    public RouteDefinition getOriginalRoute() {
+        return originalRoute;
+    }
+
+    /**
+     * Gets a list of additional tasks to execute after the {@link #configure()} method has been executed
+     * during the advice process.
+     *
+     * @return a list of additional {@link AdviceWithTask} tasks to be executed during the advice process.
+     */
+    public List<AdviceWithTask> getAdviceWithTasks() {
+        return adviceWithTasks;
+    }
+
     /**
      * Mock all endpoints in the route.
+     *
+     * @throws Exception can be thrown if error occurred
      */
     public void mockEndpoints() throws Exception {
         getContext().removeEndpoints("*");
@@ -37,7 +75,8 @@ public abstract class AdviceWithRouteBui
     /**
      * Mock all endpoints matching the given pattern.
      *
-     * @param pattern  the pattern.
+     * @param pattern the pattern.
+     * @throws Exception can be thrown if error occurred
      * @see org.apache.camel.util.EndpointHelper#matchEndpoint(String, String)
      */
     public void mockEndpoints(String pattern) throws Exception {
@@ -45,4 +84,34 @@ public abstract class AdviceWithRouteBui
         getContext().addRegisterEndpointCallback(new InterceptSendToMockEndpointStrategy(pattern));
     }
 
+    /**
+     * Advices by matching id of the nodes in the route.
+     * <p/>
+     * Uses the {@link org.apache.camel.util.EndpointHelper#matchPattern(String, String)} matching algorithm.
+     *
+     * @param pattern the pattern
+     * @return the builder
+     * @see org.apache.camel.util.EndpointHelper#matchPattern(String, String)
+     */
+    public AdviceWithBuilder adviceById(String pattern) {
+        ObjectHelper.notNull(originalRoute, "originalRoute", this);
+
+        return new AdviceWithBuilder(this, pattern, null);
+    }
+
+    /**
+     * Advices by matching the to string representation of the nodes in the route.
+     * <p/>
+     * Uses the {@link org.apache.camel.util.EndpointHelper#matchPattern(String, String)} matching algorithm.
+     *
+     * @param pattern the pattern
+     * @return the builder
+     * @see org.apache.camel.util.EndpointHelper#matchPattern(String, String)
+     */
+    public AdviceWithBuilder adviceByToString(String pattern) {
+        ObjectHelper.notNull(originalRoute, "originalRoute", this);
+
+        return new AdviceWithBuilder(this, null, pattern);
+    }
+
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTask.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTask.java?rev=1072545&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTask.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTask.java Sun Feb 20 10:22:21 2011
@@ -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.builder;
+
+/**
+ * Task or command being executed when using the advice with feature.
+ */
+public interface AdviceWithTask {
+
+    /**
+     * The task to execute
+     *
+     * @throws Exception is thrown if error during executing the task, or invalid input.
+     */
+    void task() throws Exception;
+
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTasks.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTasks.java?rev=1072545&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTasks.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/AdviceWithTasks.java Sun Feb 20 10:22:21 2011
@@ -0,0 +1,238 @@
+/**
+ * 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.builder;
+
+import java.util.Iterator;
+
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.util.EndpointHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link AdviceWithTask} tasks which are used by the {@link AdviceWithRouteBuilder}.
+ */
+public final class AdviceWithTasks {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AdviceWithTasks.class);
+
+    private AdviceWithTasks() {
+        // utility class
+    }
+
+    /**
+     * Match by is used for pluggable match by logic.
+     */
+    private interface MatchBy {
+
+        String getId();
+
+        boolean match(ProcessorDefinition processor);
+    }
+
+    /**
+     * Will match by id of the processor.
+     */
+    private static final class MatchById implements MatchBy {
+
+        private final String id;
+
+        private MatchById(String id) {
+            this.id = id;
+        }
+
+        public String getId() {
+            return id;
+        }
+
+        public boolean match(ProcessorDefinition processor) {
+            return EndpointHelper.matchPattern(processor.getId(), id);
+        }
+    }
+
+    /**
+     * Will match by the to string representation of the processor.
+     */
+    private static final class MatchByToString implements MatchBy {
+
+        private final String toString;
+
+        private MatchByToString(String toString) {
+            this.toString = toString;
+        }
+
+        public String getId() {
+            return toString;
+        }
+
+        public boolean match(ProcessorDefinition processor) {
+            return EndpointHelper.matchPattern(processor.toString(), toString);
+        }
+    }
+
+
+    public static AdviceWithTask replaceByToString(final RouteDefinition route, final String toString, final ProcessorDefinition replace) {
+        return doReplace(route, new MatchByToString(toString), replace);
+    }
+
+    public static AdviceWithTask replaceById(final RouteDefinition route, final String id, final ProcessorDefinition replace) {
+        return doReplace(route, new MatchById(id), replace);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static AdviceWithTask doReplace(final RouteDefinition route, final MatchBy matchBy, final ProcessorDefinition replace) {
+        return new AdviceWithTask() {
+            public void task() throws Exception {
+                boolean match = false;
+                Iterator<ProcessorDefinition> it = ProcessorDefinitionHelper.filterTypeInOutputs(route.getOutputs(), ProcessorDefinition.class);
+                while (it.hasNext()) {
+                    ProcessorDefinition output = it.next();
+                    if (matchBy.match(output)) {
+                        ProcessorDefinition parent = output.getParent();
+                        if (parent != null) {
+                            int index = parent.getOutputs().indexOf(output);
+                            if (index != -1) {
+                                match = true;
+                                parent.getOutputs().add(index + 1, replace);
+                                Object old = parent.getOutputs().remove(index);
+                                LOG.info("AdviceWith (" + matchBy.getId() + ") : [" + old + "] --> replace [" + replace + "]");
+                            }
+                        }
+                    }
+                }
+
+                if (!match) {
+                    throw new IllegalArgumentException("There are no outputs which matches: " + matchBy.getId() + " in the route: " + route);
+                }
+            }
+        };
+    }
+
+    public static AdviceWithTask removeByToString(final RouteDefinition route, final String toString) {
+        return doRemove(route, new MatchByToString(toString));
+    }
+
+    public static AdviceWithTask removeById(final RouteDefinition route, final String id) {
+        return doRemove(route, new MatchById(id));
+    }
+
+    @SuppressWarnings("unchecked")
+    private static AdviceWithTask doRemove(final RouteDefinition route, final MatchBy matchBy) {
+        return new AdviceWithTask() {
+            public void task() throws Exception {
+                boolean match = false;
+                Iterator<ProcessorDefinition> it = ProcessorDefinitionHelper.filterTypeInOutputs(route.getOutputs(), ProcessorDefinition.class);
+                while (it.hasNext()) {
+                    ProcessorDefinition output = it.next();
+                    if (matchBy.match(output)) {
+                        ProcessorDefinition parent = output.getParent();
+                        if (parent != null) {
+                            int index = parent.getOutputs().indexOf(output);
+                            if (index != -1) {
+                                match = true;
+                                Object old = parent.getOutputs().remove(index);
+                                LOG.info("AdviceWith (" + matchBy.getId() + ") : [" + old + "] --> remove");
+                            }
+                        }
+                    }
+                }
+
+                if (!match) {
+                    throw new IllegalArgumentException("There are no outputs which matches: " + matchBy.getId() + " in the route: " + route);
+                }
+            }
+        };
+    }
+
+    public static AdviceWithTask beforeByToString(final RouteDefinition route, final String toString, final ProcessorDefinition before) {
+        return doBefore(route, new MatchByToString(toString), before);
+    }
+
+    public static AdviceWithTask beforeById(final RouteDefinition route, final String id, final ProcessorDefinition before) {
+        return doBefore(route, new MatchById(id), before);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static AdviceWithTask doBefore(final RouteDefinition route, final MatchBy matchBy, final ProcessorDefinition before) {
+        return new AdviceWithTask() {
+            public void task() throws Exception {
+                boolean match = false;
+                Iterator<ProcessorDefinition> it = ProcessorDefinitionHelper.filterTypeInOutputs(route.getOutputs(), ProcessorDefinition.class);
+                while (it.hasNext()) {
+                    ProcessorDefinition output = it.next();
+                    if (matchBy.match(output)) {
+                        ProcessorDefinition parent = output.getParent();
+                        if (parent != null) {
+                            int index = parent.getOutputs().indexOf(output);
+                            if (index != -1) {
+                                match = true;
+                                Object existing = parent.getOutputs().get(index);
+                                parent.getOutputs().add(index, before);
+                                LOG.info("AdviceWith (" + matchBy.getId() + ") : [" + existing + "] --> before [" + before + "]");
+                            }
+                        }
+                    }
+                }
+
+                if (!match) {
+                    throw new IllegalArgumentException("There are no outputs which matches: " + matchBy.getId() + " in the route: " + route);
+                }
+            }
+        };
+    }
+
+    public static AdviceWithTask afterByToString(final RouteDefinition route, final String toString, final ProcessorDefinition after) {
+        return doAfter(route, new MatchByToString(toString), after);
+    }
+
+    public static AdviceWithTask afterById(final RouteDefinition route, final String id, final ProcessorDefinition after) {
+        return doAfter(route, new MatchById(id), after);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static AdviceWithTask doAfter(final RouteDefinition route, final MatchBy matchBy, final ProcessorDefinition after) {
+        return new AdviceWithTask() {
+            public void task() throws Exception {
+                boolean match = false;
+                Iterator<ProcessorDefinition> it = ProcessorDefinitionHelper.filterTypeInOutputs(route.getOutputs(), ProcessorDefinition.class);
+                while (it.hasNext()) {
+                    ProcessorDefinition output = it.next();
+                    if (matchBy.match(output)) {
+
+                        ProcessorDefinition parent = output.getParent();
+                        if (parent != null) {
+                            int index = parent.getOutputs().indexOf(output);
+                            if (index != -1) {
+                                match = true;
+                                Object existing = parent.getOutputs().get(index);
+                                parent.getOutputs().add(index + 1, after);
+                                LOG.info("AdviceWith (" + matchBy.getId() + ") : [" + existing + "] --> after [" + after + "]");
+                            }
+                        }
+                    }
+                }
+
+                if (!match) {
+                    throw new IllegalArgumentException("There are no outputs which matches: " + matchBy.getId() + " in the route: " + route);
+                }
+            }
+        };
+    }
+
+}

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=1072545&r1=1072544&r2=1072545&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 Sun Feb 20 10:22:21 2011
@@ -30,6 +30,9 @@ import org.apache.camel.model.OnExceptio
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.model.RoutesDefinition;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * A <a href="http://camel.apache.org/dsl.html">Java DSL</a> which is
  * used to build {@link org.apache.camel.impl.DefaultRoute} instances in a {@link CamelContext} for smart routing.
@@ -37,6 +40,7 @@ import org.apache.camel.model.RoutesDefi
  * @version 
  */
 public abstract class RouteBuilder extends BuilderSupport implements RoutesBuilder {
+    protected Logger log = LoggerFactory.getLogger(getClass());
     private AtomicBoolean initialized = new AtomicBoolean(false);
     private RoutesDefinition routeCollection = new RoutesDefinition();
 
@@ -343,6 +347,7 @@ public abstract class RouteBuilder exten
 
     /**
      * Factory method
+     *
      * @return the CamelContext
      */
     protected CamelContext createContainer() {
@@ -356,7 +361,7 @@ public abstract class RouteBuilder exten
     /**
      * Adds a collection of routes to this context
      *
-     * @param routes
+     * @param routes the routes
      * @throws Exception if the routes could not be created for whatever reason
      * @deprecated use {@link #includeRoutes(org.apache.camel.RoutesBuilder) includeRoutes} instead.
      */

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/OutputDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/OutputDefinition.java?rev=1072545&r1=1072544&r2=1072545&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/OutputDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/OutputDefinition.java Sun Feb 20 10:22:21 2011
@@ -47,4 +47,14 @@ public class OutputDefinition<Type exten
             }
         }
     }
+
+    @Override
+    public String getShortName() {
+        return "output";
+    }
+
+    @Override
+    public String toString() {
+        return getShortName() + " -> [" + outputs + "]";
+    }
 }

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=1072545&r1=1072544&r2=1072545&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 Sun Feb 20 10:22:21 2011
@@ -37,6 +37,8 @@ import org.apache.camel.Route;
 import org.apache.camel.ServiceStatus;
 import org.apache.camel.ShutdownRoute;
 import org.apache.camel.ShutdownRunningTask;
+import org.apache.camel.builder.AdviceWithRouteBuilder;
+import org.apache.camel.builder.AdviceWithTask;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.ErrorHandlerBuilderRef;
 import org.apache.camel.builder.RouteBuilder;
@@ -179,6 +181,10 @@ public class RouteDefinition extends Pro
     /**
      * Advices this route with the route builder.
      * <p/>
+     * You can use a regular {@link RouteBuilder} but the specialized {@link org.apache.camel.builder.AdviceWithRouteBuilder}
+     * has additional features when using the <a href="http://camel.apache.org/advicewith.html">advice with</a> feature.
+     * We therefore suggest you to use the {@link org.apache.camel.builder.AdviceWithRouteBuilder}.
+     * <p/>
      * The advice process will add the interceptors, on exceptions, on completions etc. configured
      * from the route builder to this route.
      * <p/>
@@ -190,14 +196,29 @@ public class RouteDefinition extends Pro
      * @param builder      the route builder
      * @return a new route which is this route merged with the route builder
      * @throws Exception can be thrown from the route builder
+     * @see AdviceWithRouteBuilder
      */
     public RouteDefinition adviceWith(CamelContext camelContext, RouteBuilder builder) throws Exception {
         ObjectHelper.notNull(camelContext, "CamelContext");
         ObjectHelper.notNull(builder, "RouteBuilder");
 
+        if (log.isDebugEnabled()) {
+            log.debug("AdviceWith route before: " + this);
+        }
+
+        // inject this route into the advice route builder so it can access this route
+        // and offer features to manipulate the route directly
+        if (builder instanceof AdviceWithRouteBuilder) {
+            ((AdviceWithRouteBuilder) builder).setOriginalRoute(this);
+        }
+
         // configure and prepare the routes from the builder
         RoutesDefinition routes = builder.configureRoutes(camelContext);
 
+        if (log.isDebugEnabled()) {
+            log.debug("AdviceWith routes: " + routes);
+        }
+
         // we can only advice with a route builder without any routes
         if (!routes.getRoutes().isEmpty()) {
             throw new IllegalArgumentException("You can only advice from a RouteBuilder which has no existing routes."
@@ -211,12 +232,23 @@ public class RouteDefinition extends Pro
         // stop and remove this existing route
         camelContext.removeRouteDefinition(this);
 
+        // any advice with tasks we should execute first?
+        if (builder instanceof AdviceWithRouteBuilder) {
+            List<AdviceWithTask> tasks = ((AdviceWithRouteBuilder) builder).getAdviceWithTasks();
+            for (AdviceWithTask task : tasks) {
+                task.task();
+            }
+        }
+
         // now merge which also ensures that interceptors and the likes get mixed in correctly as well
         RouteDefinition merged = routes.route(this);
 
         // add the new merged route
         camelContext.getRouteDefinitions().add(0, merged);
 
+        // log the merged route at info level to make it easier to end users to spot any mistakes they may have made
+        log.info("AdviceWith route after: " + merged);
+
         // and start it
         camelContext.startRoute(merged);
         return merged;

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksMatchTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksMatchTest.java?rev=1072545&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksMatchTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksMatchTest.java Sun Feb 20 10:22:21 2011
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor.interceptor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.AdviceWithRouteBuilder;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * Advice with match multiple ids test
+ */
+public class AdviceWithTasksMatchTest extends ContextTestSupport {
+
+    public void testReplaceMultipleIds() throws Exception {
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // replace all gold id's with the following route path
+                adviceById("gold*").replace().multicast().to("mock:a").to("mock:b");
+            }
+        });
+
+        getMockEndpoint("mock:foo").expectedMessageCount(0);
+        getMockEndpoint("mock:bar").expectedMessageCount(0);
+        getMockEndpoint("mock:a").expectedMessageCount(2);
+        getMockEndpoint("mock:b").expectedMessageCount(2);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .to("mock:foo").id("gold-1")
+                    .to("mock:bar").id("gold-2")
+                    .to("mock:result").id("silver-1");
+            }
+        };
+    }
+}
\ No newline at end of file

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksTest.java (from r1072268, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithRouteIdTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithRouteIdTest.java&r1=1072268&r2=1072545&rev=1072545&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithRouteIdTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksTest.java Sun Feb 20 10:22:21 2011
@@ -16,26 +16,45 @@
  */
 package org.apache.camel.processor.interceptor;
 
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.AdviceWithRouteBuilder;
 import org.apache.camel.builder.RouteBuilder;
 
 /**
- * @version 
+ * Advice with tests
  */
-public class AdviceWithRouteIdTest extends AdviceWithTest {
+public class AdviceWithTasksTest extends ContextTestSupport {
 
-    public void testAdvised() throws Exception {
-        context.getRouteDefinition("myRoute").adviceWith(context, new RouteBuilder() {
+    public void testUnknownId() throws Exception {
+        try {
+            context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+                @Override
+                public void configure() throws Exception {
+                    adviceById("xxx").replace().to("mock:xxx");
+                }
+            });
+            fail("Should hve thrown exception");
+        } catch (IllegalArgumentException e) {
+            assertTrue(e.getMessage(), e.getMessage().startsWith("There are no outputs which matches: xxx in the route"));
+        }
+    }
+
+    public void testReplace() throws Exception {
+        // START SNIPPET: e1
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
             @Override
             public void configure() throws Exception {
-                interceptSendToEndpoint("mock:foo")
-                        .skipSendToOriginalEndpoint()
-                        .to("log:foo")
-                        .to("mock:advised");
+                // advice the node in the route which has id = bar
+                // and replace it with the following route path
+                adviceById("bar").replace().multicast().to("mock:a").to("mock:b");
             }
         });
+        // END SNIPPET: e1
 
-        getMockEndpoint("mock:foo").expectedMessageCount(0);
-        getMockEndpoint("mock:advised").expectedMessageCount(1);
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        getMockEndpoint("mock:bar").expectedMessageCount(0);
+        getMockEndpoint("mock:a").expectedMessageCount(1);
+        getMockEndpoint("mock:b").expectedMessageCount(1);
         getMockEndpoint("mock:result").expectedMessageCount(1);
 
         template.sendBody("direct:start", "Hello World");
@@ -43,14 +62,80 @@ public class AdviceWithRouteIdTest exten
         assertMockEndpointsSatisfied();
     }
 
+    public void testRemove() throws Exception {
+        // START SNIPPET: e2
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // advice the node in the route which has id = bar and remove it
+                adviceById("bar").remove();
+            }
+        });
+        // END SNIPPET: e2
+
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+
+        assertFalse("Should not have removed id", context.hasEndpoint("mock:bar") == null);
+    }
+
+    public void testBefore() throws Exception {
+        // START SNIPPET: e3
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // advice the node in the route which has id = bar
+                // and insert the following route path before the adviced node
+                adviceById("bar").before().to("mock:a").transform(constant("Bye World"));
+            }
+        });
+        // END SNIPPET: e3
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:a").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Bye World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testAfter() throws Exception {
+        // START SNIPPET: e4
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // advice the node in the route which has id = bar
+                // and insert the following route path after the advice node
+                adviceById("bar").after().to("mock:a").transform(constant("Bye World"));
+            }
+        });
+        // END SNIPPET: e4
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:a").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
     @Override
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                from("direct:start").routeId("myRoute").to("mock:foo").to("mock:result");
-
-                from("direct:bar").to("mock:bar");
+                from("direct:start")
+                    .to("mock:foo")
+                    .to("mock:bar").id("bar")
+                    .to("mock:result");
             }
         };
     }

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksToStringPatternTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksToStringPatternTest.java?rev=1072545&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksToStringPatternTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/AdviceWithTasksToStringPatternTest.java Sun Feb 20 10:22:21 2011
@@ -0,0 +1,131 @@
+/**
+ * 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.interceptor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.AdviceWithRouteBuilder;
+import org.apache.camel.builder.RouteBuilder;
+
+/**
+ * Advice with using to string matching
+ */
+public class AdviceWithTasksToStringPatternTest extends ContextTestSupport {
+
+    public void testUnknownId() throws Exception {
+        try {
+            context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+                @Override
+                public void configure() throws Exception {
+                    adviceByToString("xxx").replace().to("mock:xxx");
+                }
+            });
+            fail("Should hve thrown exception");
+        } catch (IllegalArgumentException e) {
+            assertTrue(e.getMessage(), e.getMessage().startsWith("There are no outputs which matches: xxx in the route"));
+        }
+    }
+
+    public void testReplace() throws Exception {
+        // START SNIPPET: e1
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // advice nodes in the route which has foo anywhere in their to string representation
+                // and replace them with the following route path
+                adviceByToString(".*foo.*").replace().multicast().to("mock:a").to("mock:b");
+            }
+        });
+        // END SNIPPET: e1
+
+        getMockEndpoint("mock:foo").expectedMessageCount(0);
+        getMockEndpoint("mock:a").expectedMessageCount(1);
+        getMockEndpoint("mock:b").expectedMessageCount(1);
+        getMockEndpoint("mock:bar").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testRemove() throws Exception {
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                adviceByToString(".*bar.*").remove();
+            }
+        });
+
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+
+        assertFalse("Should not have removed id", context.hasEndpoint("mock:bar") == null);
+    }
+
+    public void testBefore() throws Exception {
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                adviceByToString(".*bar.*").before().to("mock:a").transform(constant("Bye World"));
+            }
+        });
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:a").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Bye World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testAfter() throws Exception {
+        context.getRouteDefinitions().get(0).adviceWith(context, new AdviceWithRouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                adviceByToString(".*bar.*").after().to("mock:a").transform(constant("Bye World"));
+            }
+        });
+
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:a").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Hello World");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Bye World");
+
+        template.sendBody("direct:start", "Hello World");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .to("mock:foo")
+                    .to("mock:bar")
+                    .to("mock:result");
+            }
+        };
+    }
+}
\ No newline at end of file