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/03/09 11:43:14 UTC

svn commit: r751648 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/processor/ camel-core/src/main/resources/org/apache/camel/model/ camel-core/src/test/java/org/apache/camel/processor/ comp...

Author: davsclaus
Date: Mon Mar  9 10:43:13 2009
New Revision: 751648

URL: http://svn.apache.org/viewvc?rev=751648&view=rev
Log:
CAMEL-1376: Introduced wireTap node in the DSL for proper wire taps.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java   (with props)
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/WireTapProcessor.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.java   (with props)
    camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringWireTapTest.java
      - copied, changed from r751602, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringMulticastTest.java
    camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringWireTapTest.xml
      - copied, changed from r751602, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/multicast.xml
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/model/InOnlyDefinition.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/SendDefinition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
    camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapTest.java
    camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/CamelContextFactoryBean.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/InOnlyDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/InOnlyDefinition.java?rev=751648&r1=751647&r2=751648&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/InOnlyDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/InOnlyDefinition.java Mon Mar  9 10:43:13 2009
@@ -57,6 +57,4 @@
     public ExchangePattern getPattern() {
         return ExchangePattern.InOnly;
     }
-
-
 }
\ 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=751648&r1=751647&r2=751648&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 Mon Mar  9 10:43:13 2009
@@ -1057,6 +1057,22 @@
     }
 
     /**
+     * <a href="http://camel.apache.org/wiretap.html">WireTap EIP:</a>
+     * Sends messages to all its child outputs; so that each processor and
+     * destination gets a copy of the original message to avoid the processors
+     * interfering with each other using {@link ExchangePattern#InOnly}.
+     *
+     * @return the builder
+     */
+    @SuppressWarnings("unchecked")
+    public Type wireTap(String uri) {
+        WireTapDefinition answer = new WireTapDefinition();
+        answer.setUri(uri);
+        addOutput(answer);
+        return (Type) this;
+    }
+
+    /**
      * Intercepts outputs added to this node in the future (i.e. intercepts outputs added after this statement)
      *
      * @param ref  a reference in the registry to lookup the interceptor that must be of type {@link DelegateProcessor}

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java?rev=751648&r1=751647&r2=751648&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java Mon Mar  9 10:43:13 2009
@@ -40,11 +40,11 @@
 @XmlAccessorType(XmlAccessType.FIELD)
 public class SendDefinition<Type extends ProcessorDefinition> extends ProcessorDefinition<Type> {
     @XmlAttribute(required = false)
-    private String uri;
+    protected String uri;
     @XmlAttribute(required = false)
-    private String ref;
+    protected String ref;
     @XmlTransient
-    private Endpoint endpoint;
+    protected Endpoint endpoint;
 
     @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java?rev=751648&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java Mon Mar  9 10:43:13 2009
@@ -0,0 +1,69 @@
+/**
+ * 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.model;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.Endpoint;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
+import org.apache.camel.processor.WireTapProcessor;
+import org.apache.camel.spi.RouteContext;
+
+/**
+ * Represents an XML &lt;wireTap/&gt; element
+ *
+ * @version $Revision$
+ */
+@XmlRootElement(name = "wireTap")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class WireTapDefinition extends SendDefinition<WireTapDefinition> {
+
+    public WireTapDefinition() {
+    }
+
+    public WireTapDefinition(String uri) {
+        setUri(uri);
+    }
+
+    public WireTapDefinition(Endpoint endpoint) {
+        setEndpoint(endpoint);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Endpoint endpoint = resolveEndpoint(routeContext);
+        return new WireTapProcessor(endpoint, getPattern());
+    }
+
+    public ExchangePattern getPattern() {
+        return ExchangePattern.InOnly;
+    }
+
+    @Override
+    public String toString() {
+        return "WireTap[" + getLabel() + "]";
+    }
+
+    @Override
+    public String getShortName() {
+        return "wireTap";
+    }
+
+}

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

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

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java?rev=751648&r1=751647&r2=751648&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java Mon Mar  9 10:43:13 2009
@@ -35,11 +35,11 @@
  * @version $Revision$
  */
 public class SendProcessor extends ServiceSupport implements AsyncProcessor, Service {
-    private static final transient Log LOG = LogFactory.getLog(SendProcessor.class);
-    private Endpoint destination;
-    private Producer producer;
-    private AsyncProcessor processor;
-    private ExchangePattern pattern;
+    protected static final transient Log LOG = LogFactory.getLog(SendProcessor.class);
+    protected Endpoint destination;
+    protected Producer producer;
+    protected AsyncProcessor processor;
+    protected ExchangePattern pattern;
 
     public SendProcessor(Endpoint destination) {
         ObjectHelper.notNull(destination, "destination");
@@ -64,7 +64,7 @@
                 throw new IllegalStateException("No producer, this processor has not been started!");
             }
         } else {
-            configureExchange(exchange);
+            exchange = configureExchange(exchange);
             producer.process(exchange);
         }
     }
@@ -79,7 +79,7 @@
             callback.done(true);
             return true;
         } else {
-            configureExchange(exchange);
+            exchange = configureExchange(exchange);
             return processor.process(exchange, callback);
         }
     }
@@ -105,9 +105,11 @@
         }
     }
 
-    protected void configureExchange(Exchange exchange) {
+    protected Exchange configureExchange(Exchange exchange) {
         if (pattern != null) {
             exchange.setPattern(pattern);
         }
+        return exchange;
     }
+
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/WireTapProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/WireTapProcessor.java?rev=751648&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/WireTapProcessor.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/WireTapProcessor.java Mon Mar  9 10:43:13 2009
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.ExchangePattern;
+
+/**
+ * Processor for wire tapping exchanges to an endpoint destination.
+ *
+ * @version $Revision$
+ */
+public class WireTapProcessor extends SendProcessor {
+
+    private int defaultThreadPoolSize = 5;
+    private ExecutorService executorService;
+
+    public WireTapProcessor(Endpoint destination) {
+        super(destination);
+    }
+
+    public WireTapProcessor(Endpoint destination, ExchangePattern pattern) {
+        super(destination, pattern);
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        if (executorService != null) {
+            executorService.shutdown();
+        }
+        super.doStop();
+    }
+
+    @Override
+    public String toString() {
+        return "wireTap(" + destination.getEndpointUri() + ")";
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        if (producer == null) {
+            if (isStopped()) {
+                LOG.warn("Ignoring exchange sent after processor is stopped: " + exchange);
+            } else {
+                throw new IllegalStateException("No producer, this processor has not been started!");
+            }
+        } else {
+            final Exchange wireTapExchange = configureExchange(exchange);
+
+            // use submit instead of execute to force it to use a new thread, execute might
+            // decide to use current thread, so we must submit a new task
+            // as we dont care for the response we dont hold the future object and wait for the result
+            getExecutorService().submit(new Callable<Object>() {
+                public Object call() throws Exception {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Processing wiretap: " + wireTapExchange);
+                    }
+                    producer.process(wireTapExchange);
+                    return null;
+                }
+            });
+        }
+    }
+
+    public boolean process(Exchange exchange, final AsyncCallback callback) {
+        if (producer == null) {
+            if (isStopped()) {
+                LOG.warn("Ignoring exchange sent after processor is stopped: " + exchange);
+            } else {
+                exchange.setException(new IllegalStateException("No producer, this processor has not been started!"));
+            }
+            callback.done(true);
+            return true;
+        } else {
+            exchange = configureExchange(exchange);
+
+            final Exchange wireTapExchange = configureExchange(exchange);
+
+            // use submit instead of execute to force it to use a new thread, execute might
+            // decide to use current thread, so we must submit a new task
+            // as we dont care for the response we dont hold the future object and wait for the result
+            getExecutorService().submit(new Callable<Object>() {
+                public Object call() throws Exception {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Processing wiretap: " + wireTapExchange);
+                    }
+                    return processor.process(wireTapExchange, callback);
+                }
+            });
+
+            // return true to indicate caller its okay, and he should not wait as this wiretap
+            // is a fire and forget
+            return true;
+        }
+    }
+
+
+    @Override
+    protected Exchange configureExchange(Exchange exchange) {
+        // must use a copy as we dont want it to cause side effects of the original exchange
+        Exchange copy = exchange.copy();
+        // set MEP to InOnly as this wire tap is a fire and forget
+        copy.setPattern(ExchangePattern.InOnly);
+        return copy;
+    }
+
+    public ExecutorService getExecutorService() {
+        if (executorService == null) {
+            executorService = createExecutorService();
+        }
+        return executorService;
+    }
+
+    private ExecutorService createExecutorService() {
+        return new ScheduledThreadPoolExecutor(defaultThreadPoolSize, new ThreadFactory() {
+            int counter;
+
+            public synchronized Thread newThread(Runnable runnable) {
+                Thread thread = new Thread(runnable);
+                thread.setName("Thread: " + (++counter) + " " + WireTapProcessor.this.toString());
+                return thread;
+            }
+        });
+    }
+
+    public void setExecutorService(ExecutorService executorService) {
+        this.executorService = executorService;
+    }
+    
+}

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

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

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=751648&r1=751647&r2=751648&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 Mon Mar  9 10:43:13 2009
@@ -69,3 +69,4 @@
 TryDefinition
 UnmarshalDefinition
 WhenDefinition
+WireTapDefinition

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapTest.java?rev=751648&r1=751647&r2=751648&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapTest.java Mon Mar  9 10:43:13 2009
@@ -20,19 +20,21 @@
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 
-import static org.apache.camel.component.mock.MockEndpoint.expectsMessageCount;
-
+/**
+ * Wire tap unit test
+ *
+ * @version $Revision$
+ */
 public class WireTapTest extends ContextTestSupport {
     protected MockEndpoint tap;
     protected MockEndpoint result;
 
     public void testSend() throws Exception {
-        String body = "<body/>";
-        tap.expectedBodiesReceived(body);
-        result.expectedBodiesReceived(body);
-        expectsMessageCount(1, tap, result);
+        // hello must come first, as we have delay on the tapped route
+        result.expectedBodiesReceived("Hello World", "Tapped");
+        tap.expectedBodiesReceived("Tapped");
 
-        template.sendBody("direct:start", body);
+        template.sendBody("direct:start", "Hello World");
 
         assertMockEndpointsSatisfied();
     }
@@ -47,8 +49,17 @@
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                from("direct:start").multicast().to("mock:tap", "mock:result");
+                // START SNIPPET: e1
+                from("direct:start")
+                    .to("log:foo")
+                    .wireTap("direct:tap")
+                    .to("mock:result");
+                // END SNIPPET: e1
+
+                from("direct:tap")
+                    .delay(100).setBody().constant("Tapped")
+                    .to("mock:result", "mock:tap");
             }
         };
     }
-}
+}
\ No newline at end of file

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.java?rev=751648&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.java Mon Mar  9 10:43:13 2009
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+import static org.apache.camel.component.mock.MockEndpoint.expectsMessageCount;
+
+public class WireTapUsingMulticastTest extends ContextTestSupport {
+    protected MockEndpoint tap;
+    protected MockEndpoint result;
+
+    public void testSend() throws Exception {
+        String body = "<body/>";
+        tap.expectedBodiesReceived(body);
+        result.expectedBodiesReceived(body);
+        expectsMessageCount(1, tap, result);
+
+        template.sendBody("direct:start", body);
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        tap = getMockEndpoint("mock:tap");
+        result = getMockEndpoint("mock:result");
+    }
+
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                from("direct:start").multicast().to("mock:tap", "mock:result");
+            }
+        };
+    }
+}

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

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/WireTapUsingMulticastTest.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=751648&r1=751647&r2=751648&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 Mon Mar  9 10:43:13 2009
@@ -34,9 +34,9 @@
 import org.apache.camel.impl.DefaultLifecycleStrategy;
 import org.apache.camel.management.DefaultInstrumentationAgent;
 import org.apache.camel.management.InstrumentationLifecycleStrategy;
-import org.apache.camel.model.OnExceptionDefinition;
 import org.apache.camel.model.IdentifiedType;
 import org.apache.camel.model.InterceptDefinition;
+import org.apache.camel.model.OnExceptionDefinition;
 import org.apache.camel.model.ProceedDefinition;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.model.RouteBuilderDefinition;

Copied: camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringWireTapTest.java (from r751602, camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringMulticastTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringWireTapTest.java?p2=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringWireTapTest.java&p1=camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringMulticastTest.java&r1=751602&r2=751648&rev=751648&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringMulticastTest.java (original)
+++ camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringWireTapTest.java Mon Mar  9 10:43:13 2009
@@ -17,14 +17,13 @@
 package org.apache.camel.spring.processor;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.processor.MulticastTest;
-
+import org.apache.camel.processor.WireTapTest;
 import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
 
-public class SpringMulticastTest extends MulticastTest {
+public class SpringWireTapTest extends WireTapTest {
 
     protected CamelContext createCamelContext() throws Exception {
-        return createSpringCamelContext(this, "org/apache/camel/spring/processor/multicast.xml");
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/SpringWireTapTest.xml");
     }
 
-}
+}
\ No newline at end of file

Copied: camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringWireTapTest.xml (from r751602, camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/multicast.xml)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringWireTapTest.xml?p2=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringWireTapTest.xml&p1=camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/multicast.xml&r1=751602&r2=751648&rev=751648&view=diff
==============================================================================
--- camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/multicast.xml (original)
+++ camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringWireTapTest.xml Mon Mar  9 10:43:13 2009
@@ -23,35 +23,23 @@
     ">
 
 
-  <!-- START SNIPPET: example -->
+    <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
+        <!-- START SNIPPET: e1 -->
+        <route>
+            <from uri="direct:start"/>
+            <to uri="log:foo"/>
+            <wireTap uri="direct:tap"/>
+            <to uri="mock:result"/>
+        </route>
+        <!-- END SNIPPET: e1 -->
 
-  <bean id="attachStringProcessor" class="org.apache.camel.processor.AppendingProcessor"/>
-
-  <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
-    <route>
-      <from uri="direct:a"/>
-      <multicast>
-          <to uri="direct:x"/>
-          <to uri="direct:y"/>
-          <to uri="direct:z"/>
-      </multicast>
-    </route>
-    <route>
-      <from uri="direct:x"/>
-      <process ref="attachStringProcessor"/>
-      <to uri="mock:x"/>
-    </route>
-    <route>
-      <from uri="direct:y"/>
-      <process ref="attachStringProcessor"/>
-      <to uri="mock:y"/>
-    </route>
-    <route>
-      <from uri="direct:z"/>
-      <process ref="attachStringProcessor"/>
-      <to uri="mock:z"/>
-    </route>
-  </camelContext>
-  <!-- END SNIPPET: example -->
+        <route>
+            <from uri="direct:tap"/>
+            <delay><constant>500</constant></delay>
+            <setBody><constant>Tapped</constant></setBody>
+            <to uri="mock:result"/>
+            <to uri="mock:tap"/>
+        </route>
+    </camelContext>
 
 </beans>