Repository: camel
Updated Branches:
  refs/heads/master 164022d25 -> 56a06b014


CAMEL-8966: Added toD as a simpler dynamic to as people often need this, and 
today they must use recipientList.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/dfe2dae8
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/dfe2dae8
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/dfe2dae8

Branch: refs/heads/master
Commit: dfe2dae8cdc61a51870ff25824a42fecd780a8b6
Parents: 164022d
Author: Claus Ibsen <davscl...@apache.org>
Authored: Mon Jul 20 11:23:27 2015 +0200
Committer: Claus Ibsen <davscl...@apache.org>
Committed: Mon Jul 20 11:54:26 2015 +0200

----------------------------------------------------------------------
 .../camel/impl/DefaultLanguageResolver.java     |   2 +
 .../apache/camel/model/DynamicToDefinition.java | 186 +++++++++++++++++
 .../apache/camel/model/ProcessorDefinition.java |  33 ++-
 .../org/apache/camel/model/ToDefinition.java    |   2 +-
 .../camel/processor/DynamicSendProcessor.java   | 208 +++++++++++++++++++
 .../apache/camel/processor/SendProcessor.java   |   4 +-
 .../resources/org/apache/camel/model/jaxb.index |   1 +
 .../management/ManagedSendProcessorTest.java    |   2 +-
 .../camel/processor/ToDynamicIgnoreTest.java    |  53 +++++
 ...amicLanguageSimpleAndXPathAndHeaderTest.java |  44 ++++
 .../ToDynamicLanguageSimpleAndXPathTest.java    |  44 ++++
 .../processor/ToDynamicLanguageXPathTest.java   |  44 ++++
 .../apache/camel/processor/ToDynamicTest.java   |  55 +++++
 .../processor/SpringToDynamicIgnoreTest.java    |  30 +++
 ...amicLanguageSimpleAndXPathAndHeaderTest.java |  30 +++
 ...ringToDynamicLanguageSimpleAndXPathTest.java |  30 +++
 .../SpringToDynamicLanguageXPathTest.java       |  31 +++
 .../spring/processor/SpringToDynamicTest.java   |  30 +++
 .../processor/SpringToDynamicIgnoreTest.xml     |  34 +++
 ...namicLanguageSimpleAndXPathAndHeaderTest.xml |  34 +++
 ...pringToDynamicLanguageSimpleAndXPathTest.xml |  34 +++
 .../SpringToDynamicLanguageXPathTest.xml        |  34 +++
 .../spring/processor/SpringToDynamicTest.xml    |  34 +++
 23 files changed, 993 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/impl/DefaultLanguageResolver.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/impl/DefaultLanguageResolver.java 
b/camel-core/src/main/java/org/apache/camel/impl/DefaultLanguageResolver.java
index 5201136..161e1d9 100644
--- 
a/camel-core/src/main/java/org/apache/camel/impl/DefaultLanguageResolver.java
+++ 
b/camel-core/src/main/java/org/apache/camel/impl/DefaultLanguageResolver.java
@@ -87,6 +87,8 @@ public class DefaultLanguageResolver implements 
LanguageResolver {
             type = findLanguageResolver("default", context);
         } catch (NoFactoryAvailableException e) {
             // ignore
+        } catch (ClassNotFoundException e) {
+            // ignore
         } catch (Exception e) {
             throw new IllegalArgumentException("Invalid URI, no 
LanguageResolver registered for scheme: " + name, e);
         }

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/model/DynamicToDefinition.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/model/DynamicToDefinition.java 
b/camel-core/src/main/java/org/apache/camel/model/DynamicToDefinition.java
new file mode 100644
index 0000000..fc1a2f7
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/DynamicToDefinition.java
@@ -0,0 +1,186 @@
+/**
+ * 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.ArrayList;
+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.ExchangePattern;
+import org.apache.camel.Expression;
+import org.apache.camel.NoSuchLanguageException;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.ExpressionBuilder;
+import org.apache.camel.processor.DynamicSendProcessor;
+import org.apache.camel.spi.Language;
+import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.ObjectHelper;
+
+/**
+ * Sends the message to a dynamic endpoint (uri supports languages)
+ * <p/>
+ * You can specify multiple languages in the uri separated by the plus sign, 
such as <tt>mock:+xpath:/order/@uri</tt>
+ * where <tt>mock:</tt> would be a prefix to a xpath expression.
+ * <p/>
+ * For more dynamic behavior use <a 
href="http://camel.apache.org/recipient-list.html";>Recipient List</a> or
+ * <a href="http://camel.apache.org/dynamic-router.html";>Dynamic Router</a> 
EIP instead.
+ */
+@Metadata(label = "eip,endpoint,routing")
+@XmlRootElement(name = "toD")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class DynamicToDefinition extends 
NoOutputDefinition<DynamicToDefinition> {
+    @XmlAttribute @Metadata(required = "true")
+    private String uri;
+    @XmlAttribute
+    private ExchangePattern pattern;
+    @XmlAttribute
+    private Integer cacheSize;
+    @XmlAttribute
+    private Boolean ignoreInvalidEndpoint;
+
+    public DynamicToDefinition() {
+    }
+
+    public DynamicToDefinition(String uri) {
+        this.uri = uri;
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws 
Exception {
+        List<Expression> list = new ArrayList<Expression>();
+        String[] parts = uri.split("\\+");
+        for (String part : parts) {
+            // the part may have optional language to use, so you can mix 
languages
+            String before = ObjectHelper.before(part, ":");
+            String after = ObjectHelper.after(part, ":");
+            if (before != null && after != null) {
+                // maybe its a language
+                try {
+                    Language partLanguage = 
routeContext.getCamelContext().resolveLanguage(before);
+                    Expression exp = partLanguage.createExpression(after);
+                    list.add(exp);
+                    continue;
+                } catch (NoSuchLanguageException e) {
+                    // ignore
+                }
+            }
+            // fallback and use simple language
+            Language lan = 
routeContext.getCamelContext().resolveLanguage("simple");
+            Expression exp = lan.createExpression(part);
+            list.add(exp);
+        }
+
+        Expression exp;
+        if (list.size() == 1) {
+            exp = list.get(0);
+        } else {
+            exp = ExpressionBuilder.concatExpression(list);
+        }
+
+        DynamicSendProcessor processor = new DynamicSendProcessor(exp);
+        processor.setPattern(pattern);
+        if (cacheSize != null) {
+            processor.setCacheSize(cacheSize);
+        }
+        if (ignoreInvalidEndpoint != null) {
+            processor.setIgnoreInvalidEndpoint(ignoreInvalidEndpoint);
+        }
+        return processor;
+    }
+
+    @Override
+    public String toString() {
+        return "DynamicTo[" + getLabel() + "]";
+    }
+
+    // Fluent API
+    // 
-------------------------------------------------------------------------
+
+    /**
+     * Sets the optional {@link ExchangePattern} used to invoke this endpoint
+     */
+    public DynamicToDefinition pattern(ExchangePattern pattern) {
+        setPattern(pattern);
+        return this;
+    }
+
+    /**
+     * Sets the maximum size used by the {@link 
org.apache.camel.impl.ConsumerCache} which is used
+     * to cache and reuse consumers when using this pollEnrich, when uris are 
reused.
+     *
+     * @param cacheSize  the cache size, use <tt>0</tt> for default cache 
size, or <tt>-1</tt> to turn cache off.
+     * @return the builder
+     */
+    public DynamicToDefinition cacheSize(int cacheSize) {
+        setCacheSize(cacheSize);
+        return this;
+    }
+
+    /**
+     * Ignore the invalidate endpoint exception when try to create a producer 
with that endpoint
+     *
+     * @return the builder
+     */
+    public DynamicToDefinition ignoreInvalidEndpoint() {
+        setIgnoreInvalidEndpoint(true);
+        return this;
+    }
+
+    // Properties
+    // 
-------------------------------------------------------------------------
+
+    public String getUri() {
+        return uri;
+    }
+
+    /**
+     * The uri of the endpoint to send to. The uri can be dynamic computed 
using the {@link org.apache.camel.language.simple.SimpleLanguage} expression.
+     */
+    public void setUri(String uri) {
+        this.uri = uri;
+    }
+
+    public ExchangePattern getPattern() {
+        return pattern;
+    }
+
+    public void setPattern(ExchangePattern pattern) {
+        this.pattern = pattern;
+    }
+
+    public Integer getCacheSize() {
+        return cacheSize;
+    }
+
+    public void setCacheSize(Integer cacheSize) {
+        this.cacheSize = cacheSize;
+    }
+
+    public Boolean getIgnoreInvalidEndpoint() {
+        return ignoreInvalidEndpoint;
+    }
+
+    public void setIgnoreInvalidEndpoint(Boolean ignoreInvalidEndpoint) {
+        this.ignoreInvalidEndpoint = ignoreInvalidEndpoint;
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java 
b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index 2359774..5d6bc36 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -606,8 +606,37 @@ public abstract class ProcessorDefinition<Type extends 
ProcessorDefinition<Type>
     public Type to(String uri) {
         addOutput(new ToDefinition(uri));
         return (Type) this;
-    }   
-    
+    }
+
+    /**
+     * Sends the exchange to the given dynamic endpoint
+     *
+     * @param uri  the dynamic endpoint to send to (resolved using simple 
language by default)
+     * @return the builder
+     */
+    @SuppressWarnings("unchecked")
+    public Type toD(String uri) {
+        DynamicToDefinition answer = new DynamicToDefinition();
+        answer.setUri(uri);
+        addOutput(answer);
+        return (Type) this;
+    }
+
+    /**
+     * Sends the exchange to the given dynamic endpoint
+     *
+     * @param uri  the dynamic endpoint to send to (resolved using simple 
language by default)
+     * @return the builder
+     */
+    @SuppressWarnings("unchecked")
+    public Type toD(String uri, boolean ignoreInvalidEndpoint) {
+        DynamicToDefinition answer = new DynamicToDefinition();
+        answer.setUri(uri);
+        answer.setIgnoreInvalidEndpoint(ignoreInvalidEndpoint);
+        addOutput(answer);
+        return (Type) this;
+    }
+
     /**
      * Sends the exchange to the given endpoint
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java 
b/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
index d3724fd..4720e98 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
@@ -26,7 +26,7 @@ import org.apache.camel.ExchangePattern;
 import org.apache.camel.spi.Metadata;
 
 /**
- * Sends the message to an endpoint
+ * Sends the message to a static endpoint
  *
  * @version 
  */

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/processor/DynamicSendProcessor.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/processor/DynamicSendProcessor.java 
b/camel-core/src/main/java/org/apache/camel/processor/DynamicSendProcessor.java
new file mode 100644
index 0000000..523ed9e
--- /dev/null
+++ 
b/camel-core/src/main/java/org/apache/camel/processor/DynamicSendProcessor.java
@@ -0,0 +1,208 @@
+/**
+ * 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.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.AsyncProducerCallback;
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Expression;
+import org.apache.camel.NoTypeConversionAvailableException;
+import org.apache.camel.Producer;
+import org.apache.camel.impl.EmptyProducerCache;
+import org.apache.camel.impl.ProducerCache;
+import org.apache.camel.spi.IdAware;
+import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.util.AsyncProcessorHelper;
+import org.apache.camel.util.EndpointHelper;
+import org.apache.camel.util.ExchangeHelper;
+import org.apache.camel.util.ServiceHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Processor for forwarding exchanges to a dynamic endpoint destination.
+ *
+ * @see org.apache.camel.processor.SendProcessor
+ */
+public class DynamicSendProcessor extends ServiceSupport implements 
AsyncProcessor, IdAware, CamelContextAware {
+    protected static final Logger LOG = 
LoggerFactory.getLogger(DynamicSendProcessor.class);
+    protected CamelContext camelContext;
+    protected final Expression expression;
+    protected ExchangePattern pattern;
+    protected ProducerCache producerCache;
+    protected String id;
+    protected boolean ignoreInvalidEndpoint;
+    protected int cacheSize;
+
+    public DynamicSendProcessor(Expression expression) {
+        this.expression = expression;
+    }
+
+    @Override
+    public String toString() {
+        return "sendTo(" + getExpression() + ")";
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public void process(final Exchange exchange) throws Exception {
+        AsyncProcessorHelper.process(this, exchange);
+    }
+
+    public boolean process(Exchange exchange, final AsyncCallback callback) {
+        if (!isStarted()) {
+            exchange.setException(new IllegalStateException("SendProcessor has 
not been started: " + this));
+            callback.done(true);
+            return true;
+        }
+
+        // we should preserve existing MEP so remember old MEP
+        // if you want to permanently to change the MEP then use 
.setExchangePattern in the DSL
+        final ExchangePattern existingPattern = exchange.getPattern();
+
+        // which endpoint to send to
+        final Endpoint endpoint;
+        final ExchangePattern destinationExchangePattern;
+
+        // use dynamic endpoint so calculate the endpoint to use
+        Object recipient = null;
+        try {
+            recipient = expression.evaluate(exchange, Object.class);
+            endpoint = resolveEndpoint(exchange, recipient);
+            destinationExchangePattern = 
EndpointHelper.resolveExchangePatternFromUrl(endpoint.getEndpointUri());
+        } catch (Throwable e) {
+            if (isIgnoreInvalidEndpoint()) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Endpoint uri is invalid: " + recipient + ". 
This exception will be ignored.", e);
+                }
+            } else {
+                exchange.setException(e);
+            }
+            callback.done(true);
+            return true;
+        }
+
+        // send the exchange to the destination using the producer cache
+        return producerCache.doInAsyncProducer(endpoint, exchange, pattern, 
callback, new AsyncProducerCallback() {
+            public boolean doInAsyncProducer(Producer producer, AsyncProcessor 
asyncProducer, final Exchange exchange,
+                                             ExchangePattern pattern, final 
AsyncCallback callback) {
+                final Exchange target = configureExchange(exchange, pattern, 
destinationExchangePattern, endpoint);
+                LOG.debug(">>>> {} {}", endpoint, exchange);
+                return asyncProducer.process(target, new AsyncCallback() {
+                    public void done(boolean doneSync) {
+                        // restore previous MEP
+                        target.setPattern(existingPattern);
+                        // signal we are done
+                        callback.done(doneSync);
+                    }
+                });
+            }
+        });
+    }
+
+    protected static Endpoint resolveEndpoint(Exchange exchange, Object 
recipient) throws NoTypeConversionAvailableException {
+        // trim strings as end users might have added spaces between separators
+        if (recipient instanceof String) {
+            recipient = ((String) recipient).trim();
+        } else if (recipient instanceof Endpoint) {
+            return (Endpoint) recipient;
+        } else {
+            // convert to a string type we can work with
+            recipient = 
exchange.getContext().getTypeConverter().mandatoryConvertTo(String.class, 
exchange, recipient);
+        }
+
+        return ExchangeHelper.resolveEndpoint(exchange, recipient);
+    }
+
+    protected Exchange configureExchange(Exchange exchange, ExchangePattern 
pattern, ExchangePattern destinationExchangePattern, Endpoint endpoint) {
+        // destination exchange pattern overrides pattern
+        if (destinationExchangePattern != null) {
+            exchange.setPattern(destinationExchangePattern);
+        } else if (pattern != null) {
+            exchange.setPattern(pattern);
+        }
+        // set property which endpoint we send to
+        exchange.setProperty(Exchange.TO_ENDPOINT, endpoint.getEndpointUri());
+        return exchange;
+    }
+
+    protected void doStart() throws Exception {
+        if (producerCache == null) {
+            if (cacheSize < 0) {
+                producerCache = new EmptyProducerCache(this, camelContext);
+                LOG.debug("DynamicSendTo {} is not using ProducerCache", this);
+            } else if (cacheSize == 0) {
+                producerCache = new ProducerCache(this, camelContext);
+                LOG.debug("DynamicSendTo {} using ProducerCache with default 
cache size", this);
+            } else {
+                producerCache = new ProducerCache(this, camelContext, 
cacheSize);
+                LOG.debug("DynamicSendTo {} using ProducerCache with 
cacheSize={}", this, cacheSize);
+            }
+        }
+    }
+
+    protected void doStop() throws Exception {
+        ServiceHelper.stopServices(producerCache);
+    }
+
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    public Expression getExpression() {
+        return expression;
+    }
+
+    public ExchangePattern getPattern() {
+        return pattern;
+    }
+
+    public void setPattern(ExchangePattern pattern) {
+        this.pattern = pattern;
+    }
+
+    public boolean isIgnoreInvalidEndpoint() {
+        return ignoreInvalidEndpoint;
+    }
+
+    public void setIgnoreInvalidEndpoint(boolean ignoreInvalidEndpoint) {
+        this.ignoreInvalidEndpoint = ignoreInvalidEndpoint;
+    }
+
+    public int getCacheSize() {
+        return cacheSize;
+    }
+
+    public void setCacheSize(int cacheSize) {
+        this.cacheSize = cacheSize;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java 
b/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
index 3ab90d6..0304af9 100644
--- a/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
+++ b/camel-core/src/main/java/org/apache/camel/processor/SendProcessor.java
@@ -46,9 +46,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Processor for forwarding exchanges to an endpoint destination.
+ * Processor for forwarding exchanges to a static endpoint destination.
  *
- * @version 
+ * @see org.apache.camel.processor.DynamicSendProcessor
  */
 public class SendProcessor extends ServiceSupport implements AsyncProcessor, 
Traceable, EndpointAware, IdAware {
     protected static final Logger LOG = 
LoggerFactory.getLogger(SendProcessor.class);

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/org/apache/camel/model/jaxb.index 
b/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
index 7bd585a..7f54628 100644
--- a/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
+++ b/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
@@ -25,6 +25,7 @@ DataFormatDefinition
 DelayDefinition
 DescriptionDefinition
 DynamicRouterDefinition
+DynamicToDefinition
 EnrichDefinition
 ExpressionSubElementDefinition
 FilterDefinition

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/management/ManagedSendProcessorTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/management/ManagedSendProcessorTest.java
 
b/camel-core/src/test/java/org/apache/camel/management/ManagedSendProcessorTest.java
index eed43c6..ad1b8a9 100644
--- 
a/camel-core/src/test/java/org/apache/camel/management/ManagedSendProcessorTest.java
+++ 
b/camel-core/src/test/java/org/apache/camel/management/ManagedSendProcessorTest.java
@@ -76,7 +76,7 @@ public class ManagedSendProcessorTest extends 
ManagementTestSupport {
 
         String json = (String) mbeanServer.invoke(on, "informationJson", null, 
null);
         assertNotNull(json);
-        assertTrue(json.contains("\"description\": \"Sends the message to an 
endpoint\""));
+        assertTrue(json.contains("\"description\": \"Sends the message to a 
static endpoint\""));
         assertTrue(json.contains(" \"uri\": { \"kind\": \"attribute\", 
\"required\": \"false\", \"type\": \"string\", \"javaType\": 
\"java.lang.String\","
                 + " \"deprecated\": \"false\", \"value\": \"mock:result\""));
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/processor/ToDynamicIgnoreTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/processor/ToDynamicIgnoreTest.java 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicIgnoreTest.java
new file mode 100644
index 0000000..9f5dfd6
--- /dev/null
+++ 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicIgnoreTest.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.processor;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+
+public class ToDynamicIgnoreTest extends ContextTestSupport {
+
+    public void testToDynamic() throws Exception {
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Hello World");
+
+        template.sendBodyAndHeader("direct:start", "Hello Camel", "foo", 
"foo");
+        template.sendBodyAndHeader("direct:start", "Hello World", "foo", 
"bar");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testToDynamicInvalid() throws Exception {
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello Camel");
+
+        template.sendBodyAndHeader("direct:start", "Hello Camel", "foo", 
"foo");
+        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")
+                    .toD("mock:${header.foo}", true);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathAndHeaderTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathAndHeaderTest.java
 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathAndHeaderTest.java
new file mode 100644
index 0000000..0c48286
--- /dev/null
+++ 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathAndHeaderTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+public class ToDynamicLanguageSimpleAndXPathAndHeaderTest extends 
ContextTestSupport {
+
+    public void testToDynamic() throws Exception {
+        getMockEndpoint("mock:foo-123").expectedBodiesReceived("<order 
uri=\"foo\"/>");
+        getMockEndpoint("mock:bar-456").expectedBodiesReceived("<order 
uri=\"bar\"/>");
+
+        template.sendBodyAndHeader("direct:start", "<order uri=\"foo\"/>", 
"sub", "-123");
+        template.sendBodyAndHeader("direct:start", "<order uri=\"bar\"/>", 
"sub", "-456");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .toD("mock:+xpath:/order/@uri+header:sub");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathTest.java
 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathTest.java
new file mode 100644
index 0000000..4827f18
--- /dev/null
+++ 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageSimpleAndXPathTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+public class ToDynamicLanguageSimpleAndXPathTest extends ContextTestSupport {
+
+    public void testToDynamic() throws Exception {
+        getMockEndpoint("mock:foo").expectedBodiesReceived("<order 
uri=\"foo\"/>");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("<order 
uri=\"bar\"/>");
+
+        template.sendBody("direct:start", "<order uri=\"foo\"/>");
+        template.sendBody("direct:start", "<order uri=\"bar\"/>");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .toD("mock:+xpath:/order/@uri");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageXPathTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageXPathTest.java
 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageXPathTest.java
new file mode 100644
index 0000000..b0ea183
--- /dev/null
+++ 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicLanguageXPathTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+public class ToDynamicLanguageXPathTest extends ContextTestSupport {
+
+    public void testToDynamic() throws Exception {
+        getMockEndpoint("mock:foo").expectedBodiesReceived("<order 
uri=\"mock:foo\"/>");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("<order 
uri=\"mock:bar\"/>");
+
+        template.sendBody("direct:start", "<order uri=\"mock:foo\"/>");
+        template.sendBody("direct:start", "<order uri=\"mock:bar\"/>");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .toD("xpath:/order/@uri");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/camel-core/src/test/java/org/apache/camel/processor/ToDynamicTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/processor/ToDynamicTest.java 
b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicTest.java
new file mode 100644
index 0000000..fc639e3
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/processor/ToDynamicTest.java
@@ -0,0 +1,55 @@
+/**
+ * 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.CamelExecutionException;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.ResolveEndpointFailedException;
+import org.apache.camel.builder.RouteBuilder;
+
+public class ToDynamicTest extends ContextTestSupport {
+
+    public void testToDynamic() throws Exception {
+        getMockEndpoint("mock:foo").expectedBodiesReceived("Hello Camel");
+        getMockEndpoint("mock:bar").expectedBodiesReceived("Hello World");
+
+        template.sendBodyAndHeader("direct:start", "Hello Camel", "foo", 
"foo");
+        template.sendBodyAndHeader("direct:start", "Hello World", "foo", 
"bar");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testToDynamicInvalid() throws Exception {
+        try {
+            template.sendBody("direct:start", "Hello Camel");
+            fail("Should fail");
+        } catch (CamelExecutionException e) {
+            assertIsInstanceOf(ResolveEndpointFailedException.class, 
e.getCause());
+        }
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .toD("mock:${header.foo}");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.java
 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.java
new file mode 100644
index 0000000..22107dd
--- /dev/null
+++ 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.java
@@ -0,0 +1,30 @@
+/**
+ * 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.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.ToDynamicIgnoreTest;
+
+import static 
org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+public class SpringToDynamicIgnoreTest extends ToDynamicIgnoreTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, 
"org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.java
 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.java
new file mode 100644
index 0000000..c4820d1
--- /dev/null
+++ 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.java
@@ -0,0 +1,30 @@
+/**
+ * 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.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.ToDynamicLanguageSimpleAndXPathAndHeaderTest;
+
+import static 
org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+public class SpringToDynamicLanguageSimpleAndXPathAndHeaderTest extends 
ToDynamicLanguageSimpleAndXPathAndHeaderTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, 
"org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.java
 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.java
new file mode 100644
index 0000000..4a751e6
--- /dev/null
+++ 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.java
@@ -0,0 +1,30 @@
+/**
+ * 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.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.ToDynamicLanguageSimpleAndXPathTest;
+
+import static 
org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+public class SpringToDynamicLanguageSimpleAndXPathTest extends 
ToDynamicLanguageSimpleAndXPathTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, 
"org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.java
 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.java
new file mode 100644
index 0000000..401f6c2
--- /dev/null
+++ 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.java
@@ -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.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.ToDynamicIgnoreTest;
+import org.apache.camel.processor.ToDynamicLanguageXPathTest;
+
+import static 
org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+public class SpringToDynamicLanguageXPathTest extends 
ToDynamicLanguageXPathTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, 
"org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicTest.java
 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicTest.java
new file mode 100644
index 0000000..37c8709
--- /dev/null
+++ 
b/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringToDynamicTest.java
@@ -0,0 +1,30 @@
+/**
+ * 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.spring.processor;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.ToDynamicTest;
+
+import static 
org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+
+public class SpringToDynamicTest extends ToDynamicTest {
+
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, 
"org/apache/camel/spring/processor/SpringToDynamicTest.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.xml
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.xml
 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.xml
new file mode 100644
index 0000000..0e8d21e
--- /dev/null
+++ 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicIgnoreTest.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans";
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring 
http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: e1 -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring";>
+    <route>
+      <from uri="direct:start"/>
+      <toD uri="mock:${header.foo}" ignoreInvalidEndpoint="true"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.xml
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.xml
 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.xml
new file mode 100644
index 0000000..64ba8f5
--- /dev/null
+++ 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathAndHeaderTest.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans";
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring 
http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: e1 -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring";>
+    <route>
+      <from uri="direct:start"/>
+      <toD uri="mock:+xpath:/order/@uri+header:sub"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.xml
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.xml
 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.xml
new file mode 100644
index 0000000..c498672
--- /dev/null
+++ 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageSimpleAndXPathTest.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans";
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring 
http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: e1 -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring";>
+    <route>
+      <from uri="direct:start"/>
+      <toD uri="mock:+xpath:/order/@uri"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.xml
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.xml
 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.xml
new file mode 100644
index 0000000..ca17736
--- /dev/null
+++ 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicLanguageXPathTest.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans";
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring 
http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: e1 -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring";>
+    <route>
+      <from uri="direct:start"/>
+      <toD uri="xpath:/order/@uri"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/dfe2dae8/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicTest.xml
----------------------------------------------------------------------
diff --git 
a/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicTest.xml
 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicTest.xml
new file mode 100644
index 0000000..0b27fbc
--- /dev/null
+++ 
b/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/SpringToDynamicTest.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans";
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring 
http://camel.apache.org/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: e1 -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring";>
+    <route>
+      <from uri="direct:start"/>
+      <toD uri="mock:${header.foo}"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

Reply via email to