You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by js...@apache.org on 2008/03/28 13:38:01 UTC

svn commit: r642209 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/builder/ main/java/org/apache/camel/component/mock/ main/java/org/apache/camel/model/ main/java/org/apache/camel/processor/ main/java/org/apache/camel/view/ main/...

Author: jstrachan
Date: Fri Mar 28 05:37:56 2008
New Revision: 642209

URL: http://svn.apache.org/viewvc?rev=642209&view=rev
Log:
patch applied from Jon Anstey with many thanks for https://issues.apache.org/activemq/browse/CAMEL-365 to implement the Routing Slip

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipType.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutingSlip.java   (with props)
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipDataModificationTest.java   (with props)
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipTest.java   (with props)
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipWithExceptionTest.java   (with props)
    activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml   (with props)
    activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml   (with props)
    activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/ProcessorBuilder.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/view/NodeData.java
    activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/ProcessorBuilder.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/ProcessorBuilder.java?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/ProcessorBuilder.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/ProcessorBuilder.java Fri Mar 28 05:37:56 2008
@@ -219,4 +219,20 @@
             }
         };
     }
+    
+    /**
+     * Throws an exception
+     */
+    public static Processor throwException(final Exception ex) {
+        return new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                throw ex;
+            }
+
+            @Override
+            public String toString() {
+                return "throwException(" + ex.toString() +  ")";
+            }
+        };
+    }
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/mock/MockEndpoint.java Fri Mar 28 05:37:56 2008
@@ -66,6 +66,9 @@
     private List expectedBodyValues;
     private List actualBodyValues;
     private PropertyChangeSupport propertyChangeSupport = new PropertyChangeSupport(this);
+    private String headerName;
+    private String headerValue;
+    private Object actualHeader;
 
     public MockEndpoint(String endpointUri, Component component) {
         super(endpointUri, component);
@@ -272,6 +275,23 @@
     }
 
     /**
+     * Adds an expectation that the given header name & value are received by this
+     * endpoint
+     */
+    public void expectedHeaderReceived(String name, String value) {
+        this.headerName = name;
+        this.headerValue = value;
+
+        expects(new Runnable() {
+            public void run() {
+                assertTrue("No header with name " + headerName + " found.", actualHeader != null);
+                
+                assertEquals("Header of message", headerValue, actualHeader);
+            }
+        });
+    }   
+    
+    /**
      * Adds an expectation that the given body values are received by this
      * endpoint
      */
@@ -539,6 +559,10 @@
     protected void performAssertions(Exchange exchange) throws Exception {
         Message in = exchange.getIn();
         Object actualBody = in.getBody();
+
+        if (headerName != null) {
+            actualHeader = in.getHeader(headerName);
+        }
 
         if (expectedBodyValues != null) {
             int index = actualBodyValues.size();

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java Fri Mar 28 05:37:56 2008
@@ -247,7 +247,7 @@
      *
      * @param coreSize the number of threads that will be used to process
      *                 messages in subsequent processors.
-     * @return a ThreadType builder that can be used to futher configure the
+     * @return a ThreadType builder that can be used to further configure the
      *         the thread pool.
      */
     public ThreadType thread(int coreSize) {
@@ -293,7 +293,7 @@
     }
 
     /**
-     * Creates a predciate expression which only if it is true then the
+     * Creates a predicate expression which only if it is true then the
      * exchange is forwarded to the destination
      *
      * @return the clause used to create the filter expression
@@ -375,7 +375,7 @@
      * href="http://activemq.apache.org/camel/recipient-list.html">Recipient
      * List</a> pattern.
      *
-     * @return the expression clasue for the expression used in the
+     * @return the expression clause for the expression used in the
      *                    {@link RecipientList} to decide the destinations
      */
     public ExpressionClause<ProcessorType<Type>> recipientList() {
@@ -387,6 +387,51 @@
     }
 
     /**
+     * Creates a <a
+     * href="http://activemq.apache.org/camel/routing-slip.html">Routing
+     * Slip</a> pattern.
+     *
+     * @param header is the header that the {@link RoutingSlip} class will
+     * look in for the list of URIs to route the message to.
+     * @param uriDelimiter is the delimiter that will be used to split up
+     * the list of URIs in the routing slip.
+     */
+    public Type routingSlip(String header, String uriDelimiter) {
+        RoutingSlipType answer = new RoutingSlipType(header, uriDelimiter);
+        addOutput(answer);
+        return (Type) this;
+    }      
+    
+    /**
+     * Creates a <a
+     * href="http://activemq.apache.org/camel/routing-slip.html">Routing
+     * Slip</a> pattern.
+     *
+     * @param header is the header that the {@link RoutingSlip} class will
+     * look in for the list of URIs to route the message to. The list of URIs
+     * will be split based on the default delimiter 
+     * {@link RoutingSlipType#DEFAULT_DELIMITER}.
+     */
+    public Type routingSlip(String header) {
+        RoutingSlipType answer = new RoutingSlipType(header);
+        addOutput(answer);
+        return (Type) this;
+    }    
+
+    /**
+     * Creates a <a
+     * href="http://activemq.apache.org/camel/routing-slip.html">Routing
+     * Slip</a> pattern with the default header {@link RoutingSlipType#ROUTING_SLIP_HEADER}.
+     * The list of URIs in the header will be split based on the default delimiter 
+     * {@link RoutingSlipType#DEFAULT_DELIMITER}.
+     */
+    public Type routingSlip() {
+        RoutingSlipType answer = new RoutingSlipType();
+        addOutput(answer);
+        return (Type) this;
+    }     
+
+    /**
      * Creates the <a
      * href="http://activemq.apache.org/camel/splitter.html">Splitter</a>
      * pattern where an expression is evaluated to iterate through each of the
@@ -846,7 +891,7 @@
      * Configures whether or not the error handler is inherited by every
      * processing node (or just the top most one)
      *
-     * @param condition the falg as to whether error handlers should be
+     * @param condition the flag as to whether error handlers should be
      *                  inherited or not
      * @return the current builder
      */

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipType.java?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipType.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/RoutingSlipType.java Fri Mar 28 05:37:56 2008
@@ -0,0 +1,86 @@
+/**
+ * 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 java.util.Collections;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.Processor;
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.processor.RoutingSlip;
+
+@XmlRootElement(name = "routingSlip")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class RoutingSlipType extends ProcessorType<ProcessorType> {
+    public static final String ROUTING_SLIP_HEADER = "routingSlipHeader";
+    public static final String DEFAULT_DELIMITER = ",";
+    
+    @XmlAttribute
+    private String headerName;
+    @XmlAttribute
+    private String uriDelimiter;
+    
+    public RoutingSlipType() {
+        this(ROUTING_SLIP_HEADER, DEFAULT_DELIMITER);       
+    }   
+    
+    public RoutingSlipType(String headerName) {
+        this(headerName, DEFAULT_DELIMITER);       
+    }
+
+    public RoutingSlipType(String headerName, String uriDelimiter) {
+        setHeaderName(headerName);
+        setUriDelimiter(uriDelimiter);
+    }
+
+    @Override
+    public String toString() {
+        return "RoutingSlip[ headerName=" + getHeaderName() 
+        + " uriDelimiter=" + getUriDelimiter() + "]";
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return new RoutingSlip(getHeaderName(), getUriDelimiter());
+    }
+    
+    @Override
+    public List<ProcessorType<?>> getOutputs() {
+        return Collections.EMPTY_LIST;
+    }
+
+    public void setHeaderName(String headerName) {
+        this.headerName = headerName;
+    }
+
+    public String getHeaderName() {
+        return this.headerName;
+    }
+
+    public void setUriDelimiter(String uriDelimiter) {
+        this.uriDelimiter = uriDelimiter;
+    }
+
+    public String getUriDelimiter() {
+        return uriDelimiter;
+    }
+}

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

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutingSlip.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutingSlip.java?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutingSlip.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutingSlip.java Fri Mar 28 05:37:56 2008
@@ -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 org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Message;
+import org.apache.camel.Processor;
+import org.apache.camel.Producer;
+import org.apache.camel.impl.MessageSupport;
+import org.apache.camel.impl.ServiceSupport;
+import org.apache.camel.model.FromType;
+import org.apache.camel.model.RoutingSlipType;
+import org.apache.camel.util.CollectionStringBuffer;
+import org.apache.camel.util.ExchangeHelper;
+import static org.apache.camel.util.ObjectHelper.notNull;
+import org.apache.camel.util.ProducerCache;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Implements a <a
+ * href="http://activemq.apache.org/camel/routing-slip.html">Routing Slip</a>
+ * pattern where the list of actual endpoints to send a message exchange to are
+ * dependent on the value of a message header.
+ * 
+ */
+public class RoutingSlip extends ServiceSupport implements Processor {
+    private static final transient Log LOG = LogFactory.getLog(RoutingSlip.class);
+	private final String header;
+    private final String uriDelimiter;
+    
+	private ProducerCache<Exchange> producerCache = new ProducerCache<Exchange>();
+
+    public RoutingSlip(String header) {
+        this(header, RoutingSlipType.DEFAULT_DELIMITER);
+    }
+
+    public RoutingSlip(String header, String uriDelimiter) {
+        notNull(header, "header");
+        notNull(uriDelimiter, "uriDelimiter");
+
+        this.header = header;
+        this.uriDelimiter = uriDelimiter;
+    }
+
+    @Override
+    public String toString() {
+        return "RoutingSlip[header=" + header + 
+        " uriDelimiter=" + uriDelimiter + "]";
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        Message message = exchange.getIn();
+        String[] recipients = recipients(message);        
+        Exchange current = exchange;
+
+        for (String nextRecipient : recipients) {            
+            Endpoint<Exchange> endpoint = resolveEndpoint(exchange, nextRecipient);
+            Producer<Exchange> producer = producerCache.getProducer(endpoint);
+            Exchange ex = endpoint.createExchange(ExchangePattern.InOut);
+
+            updateRoutingSlip(current);                    
+            copyOutToIn(ex, current);
+            
+            producer.process(ex);
+
+            current = ex;            
+        }    
+        ExchangeHelper.copyResults(exchange, current);
+    }
+
+    protected Endpoint<Exchange> resolveEndpoint(Exchange exchange, Object recipient) {
+        return ExchangeHelper.resolveEndpoint(exchange, recipient);
+    }
+
+    protected void doStop() throws Exception {
+        producerCache.stop();
+    }
+
+    protected void doStart() throws Exception {
+    }
+
+    private void updateRoutingSlip(Exchange current) {
+        Message message = getResultMessage(current);
+        message.setHeader(header, removeFirstElement(recipients(message)));
+    }
+
+    /**
+     * Returns the outbound message if available. Otherwise return the inbound message.
+     */
+    private Message getResultMessage(Exchange exchange) {
+        Message message = exchange.getOut(false);
+        // if this endpoint had no out (like a mock endpoint)
+        // just take the in
+        if (message == null) {
+            message = exchange.getIn();
+        }
+        return message;
+    }
+
+    /**
+     * Return the list of recipients defined in the routing slip in the specified message.
+     */
+    private String[] recipients(Message message) {
+        Object headerValue = message.getHeader(header);
+        if (headerValue != null && !headerValue.equals("")) {		
+            return headerValue.toString().split(uriDelimiter);
+        }
+        return new String[] {};
+    }
+
+    /**
+     * Return a string representation of the element list with the first element removed.
+     */
+    private String removeFirstElement(String[] elements) {
+        CollectionStringBuffer updatedElements = new CollectionStringBuffer(uriDelimiter);
+        for (int i = 1; i < elements.length; i++) {
+            updatedElements.append(elements[i]);
+        }
+        return updatedElements.toString();
+    }    
+    
+    /**
+     * Copy the outbound data in 'source' to the inbound data in 'result'.
+     */
+    private void copyOutToIn(Exchange result, Exchange source) {
+        result.setException(source.getException());
+        
+        Message fault = source.getFault(false);
+        if (fault != null) {
+            result.getFault(true).copyFrom(fault);
+        }
+        
+        result.setIn(getResultMessage(source));
+
+        result.getProperties().clear();
+        result.getProperties().putAll(source.getProperties());        
+    }
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/view/NodeData.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/view/NodeData.java?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/view/NodeData.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/view/NodeData.java Fri Mar 28 05:37:56 2008
@@ -28,6 +28,7 @@
 import org.apache.camel.model.OtherwiseType;
 import org.apache.camel.model.ProcessorType;
 import org.apache.camel.model.RecipientListType;
+import org.apache.camel.model.RoutingSlipType;
 import org.apache.camel.model.ResequencerType;
 import org.apache.camel.model.SplitterType;
 import org.apache.camel.model.ToType;
@@ -99,6 +100,11 @@
         } else if (node instanceof RecipientListType) {
             this.image = imagePrefix + "RecipientListIcon.png";
             this.nodeType = "Recipient List";
+	} else if (node instanceof RoutingSlipType) {            
+            this.image = imagePrefix + "RoutingSlipIcon.png";
+            this.nodeType = "Routing Slip";
+            this.url = "http://activemq.apache.org/camel/routing-slip.html";
+            this.tooltop = ((RoutingSlipType) node).getHeaderName();
         } else if (node instanceof SplitterType) {
             this.image = imagePrefix + "SplitterIcon.png";
             this.nodeType = "Splitter";

Modified: activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index (original)
+++ activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index Fri Mar 28 05:37:56 2008
@@ -40,6 +40,7 @@
 RouteType
 RouteBuilderRef
 RoutesType
+RoutingSlipType
 ServiceActivationType
 SplitterType
 ThrottlerType
@@ -48,4 +49,4 @@
 ToType
 TryType
 UnmarshalType
-WhenType
\ No newline at end of file
+WhenType

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java?rev=642209&r1=642208&r2=642209&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java Fri Mar 28 05:37:56 2008
@@ -81,6 +81,30 @@
         assertChildTo(route, "seda:b", "seda:c", "seda:d");
     }
 
+    public void testParseRoutingSlipXml() throws Exception {
+        RouteType route = assertOneRoute("routingSlip.xml");
+        assertFrom(route, "seda:a");
+        RoutingSlipType node = assertRoutingSlip(route);
+        assertEquals(RoutingSlipType.ROUTING_SLIP_HEADER, node.getHeaderName());
+        assertEquals(RoutingSlipType.DEFAULT_DELIMITER, node.getUriDelimiter());
+    }
+
+    public void testParseRoutingSlipWithHeaderSetXml() throws Exception {
+        RouteType route = assertOneRoute("routingSlipHeaderSet.xml");
+        assertFrom(route, "seda:a");
+        RoutingSlipType node = assertRoutingSlip(route);
+        assertEquals("theRoutingSlipHeader", node.getHeaderName());
+        assertEquals(RoutingSlipType.DEFAULT_DELIMITER, node.getUriDelimiter());       
+    }       
+    
+    public void testParseRoutingSlipWithHeaderAndDelimiterSetXml() throws Exception {
+        RouteType route = assertOneRoute("routingSlipHeaderAndDelimiterSet.xml");
+        assertFrom(route, "seda:a");
+        RoutingSlipType node = assertRoutingSlip(route);
+        assertEquals("theRoutingSlipHeader", node.getHeaderName());
+        assertEquals("#", node.getUriDelimiter());       
+    }   
+    
     //TODO get the test fixed
     public void xtestParseRouteWithInterceptorXml() throws Exception {
         RouteType route = assertOneRoute("routeWithInterceptor.xml");
@@ -200,6 +224,11 @@
         ProcessorType<?> processor = assertOneElement(route.getOutputs());
         return assertIsInstanceOf(RecipientListType.class, processor);
     }
+
+    protected RoutingSlipType assertRoutingSlip(ProcessorType<?> route) {
+        ProcessorType<?> processor = assertOneElement(route.getOutputs());
+        return assertIsInstanceOf(RoutingSlipType.class, processor);
+    }   
 
     protected ChoiceType assertChoice(ProcessorType<?> route) {
         ProcessorType<?> processor = assertOneElement(route.getOutputs());

Added: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipDataModificationTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipDataModificationTest.java?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipDataModificationTest.java (added)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipDataModificationTest.java Fri Mar 28 05:37:56 2008
@@ -0,0 +1,102 @@
+/**
+ * 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.routingslip;
+
+import java.util.Map;
+
+import javax.naming.Context;
+
+import org.apache.camel.Body;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Headers;
+import org.apache.camel.OutHeaders;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.util.jndi.JndiContext;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class RoutingSlipDataModificationTest extends ContextTestSupport {
+    private static final transient Log LOG = LogFactory.getLog(RoutingSlipDataModificationTest.class);
+    protected static final String ANSWER = "answer";
+    protected static final String ROUTING_SLIP_HEADER = "routingSlipHeader";
+    protected MyBean myBean = new MyBean(ROUTING_SLIP_HEADER);
+
+    public void testModificationOfDataAlongRoute()
+            throws Exception {
+        MockEndpoint x = getMockEndpoint("mock:x");
+        MockEndpoint y = getMockEndpoint("mock:y");
+        MockEndpoint z = getMockEndpoint("mock:z");
+
+        x.expectedBodiesReceived(ANSWER);
+        y.expectedBodiesReceived(ANSWER + ANSWER);
+        z.expectedBodiesReceived(ANSWER + ANSWER);
+
+        sendBody();
+
+        assertMockEndpointsSatisifed();
+    }
+    
+    protected void sendBody() {
+        template.sendBodyAndHeader("direct:a", ANSWER, ROUTING_SLIP_HEADER,
+                "mock:x,bean:myBean?method=modifyData,mock:y,mock:z");
+    }
+    
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+
+        Object lookedUpBean = context.getRegistry().lookup("myBean");
+        assertSame("Lookup of 'myBean' should return same object!", myBean, lookedUpBean);
+    }
+
+    @Override
+    protected Context createJndiContext() throws Exception {
+        JndiContext answer = new JndiContext();
+        answer.bind("myBean", myBean);
+        return answer;
+    }
+    
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // START SNIPPET: example
+                from("direct:a").routingSlip(ROUTING_SLIP_HEADER);
+                // END SNIPPET: example
+            }
+        };
+    }    
+
+    public static class MyBean {
+        private String routingSlipHeader;
+
+        public MyBean() {
+        }
+
+        public MyBean(String routingSlipHeader) {
+            this.routingSlipHeader = routingSlipHeader;
+        }
+
+        public String modifyData(
+            @Body String body, 
+            @Headers Map<String, Object> headers, 
+            @OutHeaders Map<String, Object> outHeaders) {
+            outHeaders.put(routingSlipHeader, headers.get(routingSlipHeader));
+            return body + body;
+        }
+    }    
+}

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

Added: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipTest.java?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipTest.java (added)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipTest.java Fri Mar 28 05:37:56 2008
@@ -0,0 +1,134 @@
+/**
+ * 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.routingslip;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+public class RoutingSlipTest extends ContextTestSupport {
+
+    protected static final String ANSWER = "answer";
+    protected static final String ROUTING_SLIP_HEADER = "routingSlipHeader";
+
+    public void testUpdatingOfRoutingSlipAllDefaults()
+            throws Exception {
+        MockEndpoint x = getMockEndpoint("mock:x");
+        MockEndpoint y = getMockEndpoint("mock:y");
+        MockEndpoint z = getMockEndpoint("mock:z");
+
+        // at each destination, the routing slip should contain
+        // the remaining destinations
+        x.expectedHeaderReceived(ROUTING_SLIP_HEADER, "mock:y,mock:z");
+        y.expectedHeaderReceived(ROUTING_SLIP_HEADER, "mock:z");
+        z.expectedHeaderReceived(ROUTING_SLIP_HEADER, "");
+
+        sendBody("direct:a", ROUTING_SLIP_HEADER, ",");
+
+        assertMockEndpointsSatisifed();
+    }
+
+    public void testUpdatingOfRoutingSlipHeaderSet() throws Exception {
+        MockEndpoint x = getMockEndpoint("mock:x");
+        MockEndpoint y = getMockEndpoint("mock:y");
+        MockEndpoint z = getMockEndpoint("mock:z");
+
+        // at each destination, the routing slip should contain
+        // the remaining destinations
+        x.expectedHeaderReceived("aRoutingSlipHeader", "mock:y,mock:z");
+        y.expectedHeaderReceived("aRoutingSlipHeader", "mock:z");
+        z.expectedHeaderReceived("aRoutingSlipHeader", "");
+
+        sendBody("direct:b", "aRoutingSlipHeader", ",");
+
+        assertMockEndpointsSatisifed();
+    }    
+
+    public void testUpdatingOfRoutingSlipHeaderAndDelimiterSet() throws Exception {
+        MockEndpoint x = getMockEndpoint("mock:x");
+        MockEndpoint y = getMockEndpoint("mock:y");
+        MockEndpoint z = getMockEndpoint("mock:z");
+
+        // at each destination, the routing slip should contain
+        // the remaining destinations
+        x.expectedHeaderReceived("aRoutingSlipHeader", "mock:y#mock:z");
+        y.expectedHeaderReceived("aRoutingSlipHeader", "mock:z");
+        z.expectedHeaderReceived("aRoutingSlipHeader", "");
+
+        sendBody("direct:c", "aRoutingSlipHeader", "#");
+
+        assertMockEndpointsSatisifed();
+    }       
+    
+    public void testMessagePassingThrough() throws Exception {
+        MockEndpoint end = getMockEndpoint("mock:end");
+        end.expectedMessageCount(1);
+
+        sendBody("direct:a", ROUTING_SLIP_HEADER, ",");
+
+        assertMockEndpointsSatisifed();
+    }    
+
+    public void testEmptyRoutingSlip() throws Exception {
+        MockEndpoint end = getMockEndpoint("mock:end");
+        end.expectedMessageCount(1);
+
+        sendBodyWithEmptyRoutingSlip();
+
+        assertMockEndpointsSatisifed();       
+    }
+
+    public void testNoRoutingSlip() throws Exception {
+        MockEndpoint end = getMockEndpoint("mock:end");
+        end.expectedMessageCount(1);
+
+        sendBodyWithNoRoutingSlip();
+
+        assertMockEndpointsSatisifed();       
+    }   
+    
+    protected void sendBody(String endpoint, String header, String delimiter) {
+        template.sendBodyAndHeader(endpoint, ANSWER, header,
+               "mock:x" + delimiter + "mock:y" + delimiter + "mock:z");
+    }      
+    
+    protected void sendBodyWithEmptyRoutingSlip() {
+        template.sendBodyAndHeader("direct:a", ANSWER, ROUTING_SLIP_HEADER, "");
+    }
+
+    protected void sendBodyWithNoRoutingSlip() {
+        template.sendBody("direct:a", ANSWER);
+    } 
+    
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // START SNIPPET: e1
+                from("direct:a").routingSlip().to("mock:end");
+                // END SNIPPET: e1
+                
+                // START SNIPPET: e2
+                from("direct:b").routingSlip("aRoutingSlipHeader");
+                // END SNIPPET: e2
+
+                // START SNIPPET: e3
+                from("direct:c").routingSlip("aRoutingSlipHeader", "#");
+                // END SNIPPET: e3
+            }
+        };
+    }
+}

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

Added: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipWithExceptionTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipWithExceptionTest.java?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipWithExceptionTest.java (added)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/routingslip/RoutingSlipWithExceptionTest.java Fri Mar 28 05:37:56 2008
@@ -0,0 +1,159 @@
+/**
+ * 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.routingslip;
+
+import java.util.concurrent.TimeUnit;
+
+import javax.naming.Context;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.util.jndi.JndiContext;
+
+public class RoutingSlipWithExceptionTest extends ContextTestSupport {
+
+    protected static final String ANSWER = "answer";
+    protected static final String ROUTING_SLIP_HEADER = "routingSlipHeader";
+    private MockEndpoint endEndpoint;
+    private MockEndpoint exceptionEndpoint;
+    protected MyBean myBean = new MyBean();
+    private MockEndpoint exceptionSettingEndpoint;
+
+    public void testNoException() throws Exception {
+        endEndpoint.expectedMessageCount(1);
+        exceptionEndpoint.expectedMessageCount(0);
+
+        sendRoutingSlipWithNoExceptionThrowingComponent();
+
+        assertEndpointsSatisfied();
+    }       
+
+    public void testWithExceptionThrowingComponentFirstInList() throws Exception {
+        endEndpoint.expectedMessageCount(0);
+        exceptionEndpoint.expectedMessageCount(1);
+
+        sendRoutingSlipWithExceptionThrowingComponentFirstInList();
+
+        assertEndpointsSatisfied();
+    }           
+
+    public void testWithExceptionThrowingComponentSecondInList() throws Exception {
+        endEndpoint.expectedMessageCount(0);
+        exceptionEndpoint.expectedMessageCount(1);
+
+        sendRoutingSlipWithExceptionThrowingComponentSecondInList();
+
+        assertEndpointsSatisfied();
+    }              
+    
+    public void testWithExceptionSettingComponentFirstInList() throws Exception {
+        endEndpoint.expectedMessageCount(0);
+        exceptionEndpoint.expectedMessageCount(1);      
+
+        sendRoutingSlipWithExceptionSettingComponentFirstInList();
+
+        assertEndpointsSatisfied();
+    }           
+
+    public void testWithExceptionSettingComponentSecondInList() throws Exception {
+        endEndpoint.expectedMessageCount(0);
+        exceptionEndpoint.expectedMessageCount(1);      
+
+        sendRoutingSlipWithExceptionSettingComponentSecondInList();
+
+        assertEndpointsSatisfied();
+    }
+
+    private void assertEndpointsSatisfied() throws InterruptedException {
+        MockEndpoint.assertIsSatisfied(5, TimeUnit.SECONDS, endEndpoint, exceptionEndpoint);
+    }           
+    
+    
+    protected void sendRoutingSlipWithExceptionThrowingComponentFirstInList() {
+        template.sendBodyAndHeader("direct:start", ANSWER, ROUTING_SLIP_HEADER,
+                "myBean?method=throwException,mock:x");
+    }
+
+    protected void sendRoutingSlipWithExceptionThrowingComponentSecondInList() {
+        template.sendBodyAndHeader("direct:start", ANSWER, ROUTING_SLIP_HEADER,
+                "mock:a,myBean?method=throwException");
+    }   
+    
+    protected void sendRoutingSlipWithNoExceptionThrowingComponent() {
+        template.sendBodyAndHeader("direct:start", ANSWER, ROUTING_SLIP_HEADER,
+                "mock:a");
+    }   
+
+    protected void sendRoutingSlipWithExceptionSettingComponentFirstInList() {
+        template.sendBodyAndHeader("direct:start", ANSWER, ROUTING_SLIP_HEADER,
+                "mock:exceptionSetting,mock:a");
+    }   
+
+    protected void sendRoutingSlipWithExceptionSettingComponentSecondInList() {
+        template.sendBodyAndHeader("direct:start", ANSWER, ROUTING_SLIP_HEADER,
+                "mock:a,mock:exceptionSetting");
+    }   
+       
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();    
+        
+        endEndpoint = resolveMandatoryEndpoint("mock:noexception", MockEndpoint.class);        
+        exceptionEndpoint = resolveMandatoryEndpoint("mock:exception", MockEndpoint.class); 
+        exceptionSettingEndpoint = resolveMandatoryEndpoint("mock:exceptionSetting", MockEndpoint.class);        
+
+        exceptionSettingEndpoint.whenAnyExchangeReceived(new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                exchange.setException(new Exception("Throw me!"));
+            }
+        });      
+        
+        Object lookedUpBean = context.getRegistry().lookup("myBean");
+        assertSame("Lookup of 'myBean' should return same object!", myBean, lookedUpBean);
+    }
+
+    @Override
+    protected Context createJndiContext() throws Exception {
+        JndiContext answer = new JndiContext();
+        answer.bind("myBean", myBean);
+        return answer;
+    }
+    
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                from("direct:start").
+                tryBlock().
+                  routingSlip().
+                  to("mock:noexception").
+                handle(Exception.class).to("mock:exception");
+            }
+        };
+    }    
+    
+    public static class MyBean {
+        public MyBean() {
+        }
+
+        public void throwException() throws Exception {
+            throw new Exception("Throw me!");
+        }
+    }        
+}

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

Added: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml (added)
+++ activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml Fri Mar 28 05:37:56 2008
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<routes id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+  <route>
+    <from uri="seda:a"/>
+    <routingSlip/>
+  </route>
+</routes>

Propchange: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlip.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml (added)
+++ activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml Fri Mar 28 05:37:56 2008
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<routes id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+  <route>
+    <from uri="seda:a"/>
+    <routingSlip headerName="theRoutingSlipHeader" uriDelimiter="#"/>
+  </route>
+</routes>

Propchange: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderAndDelimiterSet.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml?rev=642209&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml (added)
+++ activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml Fri Mar 28 05:37:56 2008
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+   
+    http://www.apache.org/licenses/LICENSE-2.0
+   
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<routes id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+  <route>
+    <from uri="seda:a"/>
+    <routingSlip headerName="theRoutingSlipHeader"/>
+  </route>
+</routes>

Propchange: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routingSlipHeaderSet.xml
------------------------------------------------------------------------------
    svn:eol-style = native