You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2017/03/01 08:34:31 UTC

[1/2] camel git commit: CAMEL-10908 Introduce DataTypeAware interface and let MessageSupport implement it

Repository: camel
Updated Branches:
  refs/heads/master 264ded97b -> 4d6f2a901


CAMEL-10908 Introduce DataTypeAware interface and let MessageSupport implement it

This also solves CAMEL-10890


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

Branch: refs/heads/master
Commit: 0366bc15c8ab8e6229246203c1fa4723c0f5a251
Parents: 264ded9
Author: Tomohisa Igarashi <tm...@gmail.com>
Authored: Wed Mar 1 09:06:41 2017 +0900
Committer: Tomohisa Igarashi <tm...@gmail.com>
Committed: Wed Mar 1 13:51:01 2017 +0900

----------------------------------------------------------------------
 .../java/org/apache/camel/DynamicRouter.java    |  2 +-
 .../main/java/org/apache/camel/Exchange.java    |  3 --
 .../org/apache/camel/impl/MessageSupport.java   | 28 +++++++++-
 .../apache/camel/model/InputTypeDefinition.java |  6 ++-
 .../camel/model/OutputTypeDefinition.java       |  6 ++-
 .../org/apache/camel/model/RouteDefinition.java | 38 ++++++++++---
 .../apache/camel/processor/ContractAdvice.java  | 57 ++++++++++----------
 .../camel/processor/RestBindingAdvice.java      | 27 ++++++----
 .../java/org/apache/camel/spi/DataType.java     | 11 ++++
 .../org/apache/camel/spi/DataTypeAware.java     | 51 ++++++++++++++++++
 .../java/org/apache/camel/spi/Transformer.java  | 12 ++---
 .../java/org/apache/camel/spi/Validator.java    | 16 +++---
 .../transformer/TransformerContractTest.java    |  9 +++-
 .../impl/transformer/TransformerRouteTest.java  |  3 +-
 .../impl/validator/ValidatorRouteTest.java      | 11 ++--
 .../impl/validator/SpringValidatorRouteTest.xml |  3 --
 .../transformer/demo/client/CamelClient.java    | 15 ++++--
 .../transformer/OrderRouteSpringTest.java       | 14 +++--
 18 files changed, 219 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/DynamicRouter.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/DynamicRouter.java b/camel-core/src/main/java/org/apache/camel/DynamicRouter.java
index b51ab76..d6105cd 100644
--- a/camel-core/src/main/java/org/apache/camel/DynamicRouter.java
+++ b/camel-core/src/main/java/org/apache/camel/DynamicRouter.java
@@ -42,7 +42,7 @@ import java.lang.annotation.Target;
  */
 @Retention(RetentionPolicy.RUNTIME)
 @Documented
-@Target( {ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR})
+@Target({ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR})
 public @interface DynamicRouter {
 
     /**

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/Exchange.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/Exchange.java b/camel-core/src/main/java/org/apache/camel/Exchange.java
index cc2acaa..a736174 100644
--- a/camel-core/src/main/java/org/apache/camel/Exchange.java
+++ b/camel-core/src/main/java/org/apache/camel/Exchange.java
@@ -233,9 +233,6 @@ public interface Exchange {
     String XSLT_FATAL_ERROR = "CamelXsltFatalError";
     String XSLT_WARNING     = "CamelXsltWarning";
 
-    String INPUT_TYPE  = "CamelInputType";
-    String OUTPUT_TYPE = "CamelOutputType";
-
     /**
      * Returns the {@link ExchangePattern} (MEP) of this exchange.
      *

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/impl/MessageSupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/MessageSupport.java b/camel-core/src/main/java/org/apache/camel/impl/MessageSupport.java
index a4033d0..d02fbae 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/MessageSupport.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/MessageSupport.java
@@ -20,6 +20,8 @@ import org.apache.camel.Exchange;
 import org.apache.camel.InvalidPayloadException;
 import org.apache.camel.Message;
 import org.apache.camel.TypeConverter;
+import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 
 /**
  * A base class for implementation inheritance providing the core
@@ -31,10 +33,11 @@ import org.apache.camel.TypeConverter;
  *
  * @version 
  */
-public abstract class MessageSupport implements Message {
+public abstract class MessageSupport implements Message, DataTypeAware {
     private Exchange exchange;
     private Object body;
     private String messageId;
+    private DataType dataType;
 
     @Override
     public String toString() {
@@ -112,6 +115,7 @@ public abstract class MessageSupport implements Message {
 
     public void setBody(Object body) {
         this.body = body;
+        this.dataType = body != null ? new DataType(body.getClass()) : null;
     }
 
     public <T> void setBody(Object value, Class<T> type) {
@@ -125,6 +129,25 @@ public abstract class MessageSupport implements Message {
         setBody(value);
     }
 
+    @Override
+    public void setBody(Object body, DataType type) {
+        this.body = body;
+        this.dataType = type;
+    }
+
+    @Override
+    public DataType getDataType() {
+        if (this.dataType == null) {
+            this.dataType = body != null ? new DataType(body.getClass()) : null;
+        }
+        return this.dataType;
+    }
+
+    @Override
+    public void setDataType(DataType type) {
+        this.dataType = type;
+    }
+
     public Message copy() {
         Message answer = newInstance();
         answer.copyFrom(this);
@@ -137,6 +160,9 @@ public abstract class MessageSupport implements Message {
             return;
         }
         copyFromWithNewBody(that, that.getBody());
+        if (that instanceof DataTypeAware) {
+            setDataType(((DataTypeAware)that).getDataType());
+        }
     }
 
     public void copyFromWithNewBody(Message that, Object newBody) {

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/model/InputTypeDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/InputTypeDefinition.java b/camel-core/src/main/java/org/apache/camel/model/InputTypeDefinition.java
index 7a74f8e..3ae9686 100644
--- a/camel-core/src/main/java/org/apache/camel/model/InputTypeDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/InputTypeDefinition.java
@@ -23,14 +23,16 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.camel.spi.Metadata;
 
 /**
- * Set data type of the input message.
+ * Set the expected data type of the input message. If the actual message type is different at runtime,
+ * camel look for a required {@link Transformer} and apply if exists. If validate attribute is true
+ * then camel applies {@link Validator} as well.
  * Type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
  * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
  * It's also possible to specify only scheme part, so that it works like a wildcard. If only 'xml'
  * is specified, all the XML message matches. It's handy to add only one transformer/validator
  * for all the transformation from/to XML.
  * 
- * {@see OutputTypeDefinition}
+ * @see {@link OutputTypeDefinition} {@link Transformer} {@link Validator}
  */
 @Metadata(label = "configuration")
 @XmlRootElement(name = "inputType")

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/model/OutputTypeDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/OutputTypeDefinition.java b/camel-core/src/main/java/org/apache/camel/model/OutputTypeDefinition.java
index 9862f05..c658aa4 100644
--- a/camel-core/src/main/java/org/apache/camel/model/OutputTypeDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/OutputTypeDefinition.java
@@ -23,14 +23,16 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.camel.spi.Metadata;
 
 /**
- * Sets data type of the output message.
+ * Set the expected data type of the output message. If the actual message type is different at runtime,
+ * camel look for a required {@link Transformer} and apply if exists. If validate attribute is true
+ * then camel applies {@link Validator} as well.
  * Type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
  * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
  * It's also possible to specify only scheme part, so that it works like a wildcard. If only 'xml'
  * is specified, all the XML message matches. It's handy to add only one transformer/validator
  * for all the XML-Java transformation.
  * 
- * {@see InputTypeDefinition}
+ * @see {@link InputTypeDefinition} {@link Transformer} {@link Validator}
  */
 @Metadata(label = "configuration")
 @XmlRootElement(name = "outputType")

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
index 688cb64..eee2f3b 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RouteDefinition.java
@@ -635,7 +635,10 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an input type.
+     * Declare the expected data type of the input message. If the actual message type is different
+     * at runtime, camel look for a required {@link Transformer} and apply if exists.
+     * The type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
+     * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
      * 
      * @see {@link org.apache.camel.spi.Transformer}
      * @param urn input type URN
@@ -649,7 +652,11 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an input type with validation enabled.
+     * Declare the expected data type of the input message with content validation enabled.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists, and then applies {@link Validator} as well.
+     * The type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
+     * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
      * 
      * @see {@link org.apache.camel.spi.Transformer}, {@link org.apache.camel.spi.Validator}
      * @param urn input type URN
@@ -663,7 +670,9 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an input type with Java class.
+     * Declare the expected data type of the input message by Java class.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists.
      * 
      * @see {@link org.apache.camel.spi.Transformer}
      * @param clazz Class object of the input type
@@ -677,7 +686,9 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an input type with Java class with validation enabled.
+     * Declare the expected data type of the input message by Java class with content validation enabled.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists, and then applies {@link Validator} as well.
      * 
      * @see {@link org.apache.camel.spi.Transformer}, {@link org.apache.camel.spi.Validator}
      * @param clazz Class object of the input type
@@ -691,7 +702,10 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an output type.
+     * Declare the expected data type of the output message. If the actual message type is different
+     * at runtime, camel look for a required {@link Transformer} and apply if exists.
+     * The type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
+     * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
      * 
      * @see {@link org.apache.camel.spi.Transformer}
      * @param urn output type URN
@@ -705,7 +719,11 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an output type with validation enabled.
+     * Declare the expected data type of the output message with content validation enabled.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists, and then applies {@link Validator} as well.
+     * The type name consists of two parts, 'scheme' and 'name' connected with ':'. For Java type 'name'
+     * is a fully qualified class name. For example {@code java:java.lang.String}, {@code json:ABCOrder}.
      * 
      * @see {@link org.apache.camel.spi.Transformer}, {@link org.apache.camel.spi.Validator}
      * @param urn output type URN
@@ -719,7 +737,9 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an output type with Java class.
+     * Declare the expected data type of the output message by Java class.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists.
      * 
      * @see {@link org.apache.camel.spi.Transformer}
      * @param clazz Class object of the output type
@@ -733,7 +753,9 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     }
 
     /**
-     * Declare an output type with Java class with validation enabled.
+     * Declare the expected data type of the ouput message by Java class with content validation enabled.
+     * If the actual message type is different at runtime, camel look for a required
+     * {@link Transformer} and apply if exists, and then applies {@link Validator} as well.
      * 
      * @see {@link org.apache.camel.spi.Transformer}, {@link org.apache.camel.spi.Validator}
      * @param clazz Class object of the output type

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/processor/ContractAdvice.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/processor/ContractAdvice.java b/camel-core/src/main/java/org/apache/camel/processor/ContractAdvice.java
index 6a53e7b..361b47a 100644
--- a/camel-core/src/main/java/org/apache/camel/processor/ContractAdvice.java
+++ b/camel-core/src/main/java/org/apache/camel/processor/ContractAdvice.java
@@ -22,16 +22,25 @@ import org.apache.camel.Message;
 import org.apache.camel.ValidationException;
 import org.apache.camel.spi.Contract;
 import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.apache.camel.spi.Transformer;
 import org.apache.camel.spi.Validator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * A {@code CamelInternalProcessorAdvice} which performs Transformation and Validation
+ * A {@link CamelInternalProcessorAdvice} which applies {@link Transformer} and {@link Validator}
  * according to the data type Contract.
+ * The default camel {@link Message} implements {@link DataTypeAware} which
+ * holds a {@link DataType} to indicate current message type. If the input type
+ * declared by {@link InputTypeDefinition} is diffrent from current IN message type,
+ * camel internal processor look for a Transformer which transforms from the current
+ * message type to the expected message type before routing.
+ * After routing, if the output type declared by {@link OutputTypeDefinition} is different
+ * from current OUT message (or IN message if no OUT), camel look for a Transformer and apply.
  * 
- * @see CamelInternalProcessor, CamelInternalProcessorAdvice
+ * @see {@link Transformer} {@link Validator}
+ * {@link InputTypeDefinition} {@link OutputTypeDefinition}
  */
 public class ContractAdvice implements CamelInternalProcessorAdvice {
     private static final Logger LOG = LoggerFactory.getLogger(CamelInternalProcessor.class);
@@ -44,13 +53,17 @@ public class ContractAdvice implements CamelInternalProcessorAdvice {
     
     @Override
     public Object before(Exchange exchange) throws Exception {
-        DataType from = getCurrentType(exchange, Exchange.INPUT_TYPE);
+        if (!(exchange.getIn() instanceof DataTypeAware)) {
+            return null;
+        }
+        DataTypeAware target = (DataTypeAware)exchange.getIn();
+        DataType from = target.getDataType();
         DataType to = contract.getInputType();
         if (to != null) {
             if (!to.equals(from)) {
                 LOG.debug("Looking for transformer for INPUT: from='{}', to='{}'", from, to);
                 doTransform(exchange.getIn(), from, to);
-                exchange.setProperty(Exchange.INPUT_TYPE, to);
+                target.setDataType(to);
             }
             if (contract.isValidateInput()) {
                 doValidate(exchange.getIn(), to);
@@ -65,18 +78,19 @@ public class ContractAdvice implements CamelInternalProcessorAdvice {
             // TODO can we add FAULT_TYPE processing?
             return;
         }
-        
+
         Message target = exchange.hasOut() ? exchange.getOut() : exchange.getIn();
-        if (!exchange.hasOut() && exchange.getProperty(Exchange.OUTPUT_TYPE) == null) {
-            exchange.setProperty(Exchange.OUTPUT_TYPE, exchange.getProperty(Exchange.INPUT_TYPE));
+        if (!(target instanceof DataTypeAware)) {
+            return;
         }
-        DataType from = getCurrentType(exchange, Exchange.OUTPUT_TYPE);
+        DataTypeAware typeAwareTarget = (DataTypeAware)target;
+        DataType from = typeAwareTarget.getDataType();
         DataType to = contract.getOutputType();
         if (to != null) {
             if (!to.equals(from)) {
                 LOG.debug("Looking for transformer for OUTPUT: from='{}', to='{}'", from, to);
                 doTransform(target, from, to);
-                exchange.setProperty(Exchange.OUTPUT_TYPE, to);
+                typeAwareTarget.setDataType(to);
             }
             if (contract.isValidateOutput()) {
                 doValidate(target, to);
@@ -87,7 +101,7 @@ public class ContractAdvice implements CamelInternalProcessorAdvice {
     private void doTransform(Message message, DataType from, DataType to) throws Exception {
         if (from == null) {
             // If 'from' is null, only Java-Java convertion is performed.
-            // It means if 'to' is other than Java, it's assumed to be already in expected shape.
+            // It means if 'to' is other than Java, it's assumed to be already in expected type.
             convertIfRequired(message, to);
             return;
         }
@@ -99,10 +113,11 @@ public class ContractAdvice implements CamelInternalProcessorAdvice {
             // Found matched transformer. Java-Java transformer is also allowed.
             return;
         } else if (from.isJavaType()) {
-            if (convertIfRequired(message, to)) {
-                // Java->Java transformation just relies on TypeConverter if no explicit transformer
-                return;
-            }
+            // Try TypeConverter as a fallback for Java->Java transformation
+            convertIfRequired(message, to);
+            // If Java->Other transformation required but no transformer matched,
+            // then assume it's already in expected type, i.e. do nothing.
+            return;
         } else if (applyTransformerChain(message, from, to)) {
             // Other->Other transformation - found a transformer chain
             return;
@@ -152,22 +167,10 @@ public class ContractAdvice implements CamelInternalProcessorAdvice {
         }
         return false;
     }
-    
+
     private Class<?> getClazz(String type, CamelContext context) throws Exception {
         return context.getClassResolver().resolveMandatoryClass(type);
     }
-    
-    private DataType getCurrentType(Exchange exchange, String name) {
-        Object prop = exchange.getProperty(name);
-        if (prop instanceof DataType) {
-            return (DataType)prop;
-        } else if (prop instanceof String) {
-            DataType answer = new DataType((String)prop);
-            exchange.setProperty(name, answer);
-            return answer;
-        }
-        return null;
-    }
 
     private void doValidate(Message message, DataType type) throws ValidationException {
         Validator validator = message.getExchange().getContext().resolveValidator(type);

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/processor/RestBindingAdvice.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/processor/RestBindingAdvice.java b/camel-core/src/main/java/org/apache/camel/processor/RestBindingAdvice.java
index 7b81a19..d14c0f4 100644
--- a/camel-core/src/main/java/org/apache/camel/processor/RestBindingAdvice.java
+++ b/camel-core/src/main/java/org/apache/camel/processor/RestBindingAdvice.java
@@ -29,6 +29,7 @@ import org.apache.camel.processor.binding.BindingException;
 import org.apache.camel.spi.Contract;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.Transformer;
 import org.apache.camel.util.ExchangeHelper;
@@ -167,11 +168,8 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
             isJson = consumes != null && consumes.toLowerCase(Locale.ENGLISH).contains("json");
         }
 
-        // set the INPUT_TYPE to indicate body type
-        if (isJson) {
-            exchange.setProperty(Exchange.INPUT_TYPE, new DataType("json"));
-        } else if (isXml) {
-            exchange.setProperty(Exchange.INPUT_TYPE, new DataType("xml"));
+        if (exchange.getIn() instanceof DataTypeAware && (isJson || isXml)) {
+            ((DataTypeAware)exchange.getIn()).setDataType(new DataType(isJson ? "json" : "xml"));
         }
 
         // only allow xml/json if the binding mode allows that
@@ -196,7 +194,11 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
                 // so force reading the body as a String which we can work with
                 body = MessageHelper.extractBodyAsString(exchange.getIn());
                 if (body != null) {
-                    exchange.getIn().setBody(body);
+                    if (exchange.getIn() instanceof DataTypeAware) {
+                        ((DataTypeAware)exchange.getIn()).setBody(body, new DataType(isJson ? "json" : "xml"));
+                    } else {
+                        exchange.getIn().setBody(body);
+                    }
 
                     if (isXml && isJson) {
                         // we have still not determined between xml or json, so check the body if its xml based or not
@@ -223,7 +225,6 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
             if (ObjectHelper.isNotEmpty(body)) {
                 jsonUnmarshal.process(exchange);
                 ExchangeHelper.prepareOutToIn(exchange);
-                exchange.setProperty(Exchange.INPUT_TYPE, new DataType(exchange.getIn().getBody().getClass()));
             }
             return;
         } else if (isXml && xmlUnmarshal != null) {
@@ -232,7 +233,6 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
             if (ObjectHelper.isNotEmpty(body)) {
                 xmlUnmarshal.process(exchange);
                 ExchangeHelper.prepareOutToIn(exchange);
-                exchange.setProperty(Exchange.INPUT_TYPE, new DataType(exchange.getIn().getBody().getClass()));
             }
             return;
         }
@@ -337,13 +337,13 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
                 // only marshal if its json content type
                 if (contentType.contains("json")) {
                     jsonMarshal.process(exchange);
-                    exchange.setProperty(Exchange.OUTPUT_TYPE, new DataType("json"));
+                    setOutputDataType(exchange, new DataType("json"));
                 }
             } else if (isXml && xmlMarshal != null) {
                 // only marshal if its xml content type
                 if (contentType.contains("xml")) {
                     xmlMarshal.process(exchange);
-                    exchange.setProperty(Exchange.OUTPUT_TYPE, new DataType("xml"));
+                    setOutputDataType(exchange, new DataType("xml"));
                 }
             } else {
                 // we could not bind
@@ -362,6 +362,13 @@ public class RestBindingAdvice implements CamelInternalProcessorAdvice<Map<Strin
         }
     }
 
+    private void setOutputDataType(Exchange exchange, DataType type) {
+        Message target = exchange.hasOut() ? exchange.getOut() : exchange.getIn();
+        if (target instanceof DataTypeAware) {
+            ((DataTypeAware)target).setDataType(type);
+        }
+    }
+
     private void ensureHeaderContentType(String contentType, boolean isXml, boolean isJson, Exchange exchange) {
         // favor given content type
         if (contentType != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/spi/DataType.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/DataType.java b/camel-core/src/main/java/org/apache/camel/spi/DataType.java
index 973e5c4..7c07188 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/DataType.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/DataType.java
@@ -20,6 +20,17 @@ import org.apache.camel.util.StringHelper;
 
 /**
  * Represents the data type URN which is used for message data type contract.
+ * Java class doesn't always explain the data type completely, for example XML and JSON
+ * data format is sometimes serialized as a {@code String}, {@code InputStream} or etc.
+ * The {@link DataTypeAware} message stores the DataType as a part of the message to carry
+ * those data type information even if it's marshaled, so that it could be
+ * leveraged to detect required {@link Transformer} and {@link Validator}.
+ * DataType consists of two parts, 'model' and 'name'. Its string representation is
+ * 'model:name' connected with colon. For example 'java:com.example.Order', 'xml:ABCOrder'
+ * or 'json:XYZOrder'. These type name other than java class name allows the message to
+ * carry the name of the message data structure even if it's marshaled.
+ * 
+ * @see {@link DataTypeAware} {@link Transformer} {@link Validator}
  */
 public class DataType {
 

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/spi/DataTypeAware.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/DataTypeAware.java b/camel-core/src/main/java/org/apache/camel/spi/DataTypeAware.java
new file mode 100644
index 0000000..f979f74
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/DataTypeAware.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.spi;
+
+/**
+ * Allows {@link org.apache.camel.Message} to store a {@link DataType} which
+ * represents the data type of the Message. Sometimes message content is marshaled
+ * into {@code String}, {@code InputStream} or etc, and the data type structure is
+ * not available until it's unmarshaled into Java object. The {@link DataType} stored
+ * in a DataTypeAware message carries that missing data type information even if it's
+ * marshaled, and whatever the Java class of the body is. This type information is used
+ * to detect required {@link Transformer} and {@link Validator}.
+ * 
+ * @see {@link DataType} {@link Transformer} {@link Validator}
+ */
+public interface DataTypeAware {
+
+    /**
+     * Set the data type of the message.
+     * @param type data type
+     */
+    void setDataType(DataType type);
+
+    /**
+     * Get the data type of the message.
+     * @return data type
+     */
+    DataType getDataType();
+
+    /**
+     * Set the message body with data type.
+     * @param body message body
+     * @param type data type
+     */
+    void setBody(Object body, DataType type);
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/spi/Transformer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/Transformer.java b/camel-core/src/main/java/org/apache/camel/spi/Transformer.java
index 23439f5..32fd376 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/Transformer.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/Transformer.java
@@ -27,15 +27,11 @@ import org.apache.camel.support.ServiceSupport;
 /**
  * <a href="http://camel.apache.org/transformer.html">Transformer</a>
  * performs message transformation according to the declared data type.
- * There are two Exchange property indicates current message type, {@link Exchange#INPUT_TYPE}
- * holds input message type and {@link Exchange#OUTPUT_TYPE} holds output message type. If the
- * input type and/or output type declared by {@link InputTypeDefinition}
- * and/or {@link OutputTypeDefinition} in the route definition is different from those property
- * at runtime, camel internal processor look for a Transformer which transforms from
- * the current message type to the expected message type.
+ * {@link ContractAdvice} looks for a required Transformer and apply if
+ * input/output type declared on a route is different from current message type.
  *  
- * @see InputTypeDefinition
- * @see OutputTypeDefinition
+ * @see {@link ContractAdvice}
+ * {@link DataType} {@link InputTypeDefinition} {@link OutputTypeDefinition}
  */
 public abstract class Transformer extends ServiceSupport implements CamelContextAware {
 

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/main/java/org/apache/camel/spi/Validator.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/Validator.java b/camel-core/src/main/java/org/apache/camel/spi/Validator.java
index 1ba2998..ebd5776 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/Validator.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/Validator.java
@@ -26,17 +26,13 @@ import org.apache.camel.model.OutputTypeDefinition;
 import org.apache.camel.support.ServiceSupport;
 
 /**
- * <a href="http://camel.apache.org/transformer.html">Transformer</a>
- * performs message transformation according to the declared data type.
- * There are two Exchange property indicates current message type, {@link Exchange#INPUT_TYPE}
- * holds input message type and {@link Exchange#OUTPUT_TYPE} holds output message type. If the
- * input type and/or output type declared by {@link InputTypeDefinition}
- * and/or {@link OutputTypeDefinition} in the route definition is different from those property
- * at runtime, camel internal processor look for a Transformer which transforms from
- * the current message type to the expected message type.
+ * <a href="http://camel.apache.org/validator.html">Validator</a>
+ * performs message content validation according to the declared data type.
+ * {@link ContractAdvice} applies Validator if input/output type is declared on
+ * a route with validation enabled.
  *  
- * @see InputTypeDefinition
- * @see OutputTypeDefinition
+ * @see {@link ContractAdvice}
+ * {@link InputTypeDefinition} {@link OutputTypeDefinition}
  */
 public abstract class Validator extends ServiceSupport implements CamelContextAware {
 

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerContractTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerContractTest.java b/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerContractTest.java
index 56cf91d..7458e5c 100644
--- a/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerContractTest.java
+++ b/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerContractTest.java
@@ -30,6 +30,8 @@ import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 import org.apache.camel.model.DataFormatDefinition;
 import org.apache.camel.spi.DataFormat;
+import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.apache.camel.spi.RouteContext;
 import org.junit.Test;
 
@@ -115,7 +117,10 @@ public class TransformerContractTest extends ContextTestSupport {
         mocka.setExpectedCount(1);
         mocka2.setExpectedCount(1);
         mockb.setExpectedCount(1);
-        Object answer = template.requestBody("direct:a", "<foo/>");
+        Exchange answer = template.send("direct:a", ex -> {
+            DataTypeAware message = (DataTypeAware)ex.getIn();
+            message.setBody("<foo/>", new DataType("xml"));
+        });
         mocka.assertIsSatisfied();
         mocka2.assertIsSatisfied();
         mockb.assertIsSatisfied();
@@ -125,7 +130,7 @@ public class TransformerContractTest extends ContextTestSupport {
         assertEquals("<foo/>", exa.getIn().getBody());
         assertEquals(A.class, exb.getIn().getBody().getClass());
         assertEquals(B.class, exa2.getIn().getBody().getClass());
-        assertEquals("<fooResponse/>", new String((byte[])answer));
+        assertEquals("<fooResponse/>", new String((byte[])answer.getIn().getBody()));
     }
 
     public static class MyTypeConverters implements TypeConverters {

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerRouteTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerRouteTest.java b/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerRouteTest.java
index 4d5a0d1..8a07c4a 100644
--- a/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerRouteTest.java
+++ b/camel-core/src/test/java/org/apache/camel/impl/transformer/TransformerRouteTest.java
@@ -42,6 +42,7 @@ import org.apache.camel.impl.DefaultExchange;
 import org.apache.camel.model.DataFormatDefinition;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.Transformer;
 import org.slf4j.Logger;
@@ -98,7 +99,7 @@ public class TransformerRouteTest extends ContextTestSupport {
         });
 
         Exchange exchange = new DefaultExchange(context, ExchangePattern.InOut);
-        exchange.getIn().setBody("{name:XOrder}");
+        ((DataTypeAware)exchange.getIn()).setBody("{name:XOrder}", new DataType("json:JsonXOrder"));
         Exchange answerEx = template.send("direct:dataFormat", exchange);
         if (answerEx.getException() != null) {
             throw answerEx.getException();

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/camel-core/src/test/java/org/apache/camel/impl/validator/ValidatorRouteTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/validator/ValidatorRouteTest.java b/camel-core/src/test/java/org/apache/camel/impl/validator/ValidatorRouteTest.java
index 3d92741..696881d 100644
--- a/camel-core/src/test/java/org/apache/camel/impl/validator/ValidatorRouteTest.java
+++ b/camel-core/src/test/java/org/apache/camel/impl/validator/ValidatorRouteTest.java
@@ -64,7 +64,7 @@ public class ValidatorRouteTest extends ContextTestSupport {
         if (answerEx.getException() != null) {
             throw answerEx.getException();
         }
-        assertEquals("{name:XOrderResponse}", answerEx.getOut().getBody(String.class));
+        assertEquals("{name:XOrderResponse}", answerEx.getIn().getBody(String.class));
     }
 
     public void testEndpointValidator() throws Exception {
@@ -101,8 +101,7 @@ public class ValidatorRouteTest extends ContextTestSupport {
                 from("direct:predicate")
                     .inputTypeWithValidate("json:JsonXOrder")
                     .outputType("json:JsonXOrderResponse")
-                    .setBody(simple("{name:XOrderResponse}"))
-                    .setProperty(Exchange.OUTPUT_TYPE, constant("json:JsonXOrderResponse"));
+                    .setBody(simple("{name:XOrderResponse}"));
                 
                 context.addComponent("myxml", new MyXmlComponent());
                 validator()
@@ -112,8 +111,7 @@ public class ValidatorRouteTest extends ContextTestSupport {
                     .inputType("xml:XmlXOrder")
                     .outputTypeWithValidate("xml:XmlXOrderResponse")
                     .validate(exchangeProperty(VALIDATOR_INVOKED).isNull())
-                    .setBody(simple("<XOrderResponse/>"))
-                    .setProperty(Exchange.OUTPUT_TYPE, constant("xml:XmlXOrderResponse"));
+                    .setBody(simple("<XOrderResponse/>"));
                 
                 validator()
                     .type("other:OtherXOrder")
@@ -125,8 +123,7 @@ public class ValidatorRouteTest extends ContextTestSupport {
                     .inputTypeWithValidate("other:OtherXOrder")
                     .outputTypeWithValidate("other:OtherXOrderResponse")
                     .validate(exchangeProperty(VALIDATOR_INVOKED).isEqualTo(OtherXOrderValidator.class))
-                    .setBody(simple("name=XOrderResponse"))
-                    .setProperty(Exchange.OUTPUT_TYPE, constant("other:OtherXOrderResponse"));
+                    .setBody(simple("name=XOrderResponse"));
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/components/camel-spring/src/test/resources/org/apache/camel/spring/impl/validator/SpringValidatorRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-spring/src/test/resources/org/apache/camel/spring/impl/validator/SpringValidatorRouteTest.xml b/components/camel-spring/src/test/resources/org/apache/camel/spring/impl/validator/SpringValidatorRouteTest.xml
index 4e6c090..fce29a6 100644
--- a/components/camel-spring/src/test/resources/org/apache/camel/spring/impl/validator/SpringValidatorRouteTest.xml
+++ b/components/camel-spring/src/test/resources/org/apache/camel/spring/impl/validator/SpringValidatorRouteTest.xml
@@ -38,7 +38,6 @@
             <inputType urn="json:JsonXOrder" validate="true"/>
             <outputType urn="json:JsonXOrderResponse"/>
             <setBody><constant>{name:XOrderResponse}</constant></setBody>
-            <setProperty propertyName="CamelOutputType"><constant>json:JsonXOrderResponse</constant></setProperty>
         </route>
 
         <route>
@@ -49,7 +48,6 @@
                 <simple>${exchangeProperty.validator-invoked} == null</simple>
             </validate>
             <setBody><constant>&lt;XOrderResponse/&gt;</constant></setBody>
-            <setProperty propertyName="CamelOutputType"><constant>xml:XmlXOrderResponse</constant></setProperty>
         </route>
         
         <route>
@@ -60,7 +58,6 @@
                 <simple>${exchangeProperty.validator-invoked} == 'org.apache.camel.impl.validator.ValidatorRouteTest$OtherXOrderValidator'</simple>
             </validate>
             <setBody><constant>name=XOrderResponse</constant></setBody>
-            <setProperty propertyName="CamelOutputType"><constant>other:OtherXOrderResponse</constant></setProperty>
         </route>
         
     </camelContext>

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/examples/camel-example-transformer-demo/src/main/java/org/apache/camel/example/transformer/demo/client/CamelClient.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-transformer-demo/src/main/java/org/apache/camel/example/transformer/demo/client/CamelClient.java b/examples/camel-example-transformer-demo/src/main/java/org/apache/camel/example/transformer/demo/client/CamelClient.java
index 2f42ef8..346579d 100644
--- a/examples/camel-example-transformer-demo/src/main/java/org/apache/camel/example/transformer/demo/client/CamelClient.java
+++ b/examples/camel-example-transformer-demo/src/main/java/org/apache/camel/example/transformer/demo/client/CamelClient.java
@@ -21,9 +21,12 @@ import java.io.File;
 import java.io.FileReader;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
 import org.apache.camel.ProducerTemplate;
 import org.apache.camel.example.transformer.demo.Order;
 import org.apache.camel.example.transformer.demo.OrderResponse;
+import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
@@ -68,17 +71,21 @@ public final class CamelClient {
         
         String orderXml = "<order orderId=\"Order-XML-0001\" itemId=\"MIKAN\" quantity=\"365\"/>";
         LOG.info("---> Sending '{}' to 'direct:xml'", orderXml);
-        String responseXml = producer.requestBody("direct:xml", orderXml, String.class);
+        Exchange answerXml = producer.send("direct:xml", ex -> {
+            ((DataTypeAware)ex.getIn()).setBody(orderXml, new DataType("xml:XMLOrder"));
+        });
         Thread.sleep(1000);
-        LOG.info("---> Received '{}'", responseXml);
+        LOG.info("---> Received '{}'", answerXml.getOut().getBody(String.class));
         LOG.info("---> CSV log now contains:{}", getCsvLog());
         Thread.sleep(1000);
         
         String orderJson = "{\"orderId\":\"Order-JSON-0001\", \"itemId\":\"MIZUYO-KAN\", \"quantity\":\"16350\"}";
         LOG.info("---> Sending '{}' to 'direct:json'", orderJson);
-        String responseJson = producer.requestBody("direct:json", orderJson, String.class);
+        Exchange answerJson = producer.send("direct:json", ex -> {
+            ((DataTypeAware)ex.getIn()).setBody(orderJson, new DataType("json"));
+        });
         Thread.sleep(1000);
-        LOG.info("---> Received '{}'", responseJson);
+        LOG.info("---> Received '{}'", answerJson.getOut().getBody(String.class));
         LOG.info("---> CSV log now contains:{}", getCsvLog());
         Thread.sleep(1000);
         

http://git-wip-us.apache.org/repos/asf/camel/blob/0366bc15/examples/camel-example-transformer-demo/src/test/java/org/apache/camel/example/transformer/OrderRouteSpringTest.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-transformer-demo/src/test/java/org/apache/camel/example/transformer/OrderRouteSpringTest.java b/examples/camel-example-transformer-demo/src/test/java/org/apache/camel/example/transformer/OrderRouteSpringTest.java
index 44d287c..8d036aa 100644
--- a/examples/camel-example-transformer-demo/src/test/java/org/apache/camel/example/transformer/OrderRouteSpringTest.java
+++ b/examples/camel-example-transformer-demo/src/test/java/org/apache/camel/example/transformer/OrderRouteSpringTest.java
@@ -25,6 +25,8 @@ import org.apache.camel.ProducerTemplate;
 import org.apache.camel.component.mock.MockEndpoint;
 import org.apache.camel.example.transformer.demo.Order;
 import org.apache.camel.example.transformer.demo.OrderResponse;
+import org.apache.camel.spi.DataType;
+import org.apache.camel.spi.DataTypeAware;
 import org.apache.camel.test.spring.CamelSpringDelegatingTestContextLoader;
 import org.apache.camel.test.spring.CamelSpringRunner;
 import org.apache.camel.test.spring.MockEndpointsAndSkip;
@@ -100,8 +102,10 @@ public class OrderRouteSpringTest {
 
         String order = "<order orderId=\"Order-XML-0001\" itemId=\"MIKAN\" quantity=\"365\"/>";
         String expectedAnswer = "<orderResponse orderId=\"Order-XML-0001\" accepted=\"true\" description=\"Order accepted:[item='MIKAN' quantity='365']\"/>";
-        String answer = xmlProducer.requestBody("direct:xml", order, String.class);
-        XMLUnit.compareXML(expectedAnswer, answer);
+        Exchange answer = xmlProducer.send("direct:xml", ex -> {
+            ((DataTypeAware)ex.getIn()).setBody(order, new DataType("xml:XMLOrder"));
+        });
+        XMLUnit.compareXML(expectedAnswer, answer.getOut().getBody(String.class));
         mockCsv.assertIsSatisfied();
     }
 
@@ -126,8 +130,10 @@ public class OrderRouteSpringTest {
             .setDescription("Order accepted:[item='MIZUYO-KAN' quantity='16350']");
         ObjectMapper jsonMapper = new ObjectMapper();
         String expectedJson = jsonMapper.writeValueAsString(expected);
-        String answer = jsonProducer.requestBody("direct:json", order, String.class);
-        assertEquals(expectedJson, answer);
+        Exchange answer = jsonProducer.send("direct:json", ex -> {
+            ((DataTypeAware)ex.getIn()).setBody(order, new DataType("json"));
+        });
+        assertEquals(expectedJson, answer.getOut().getBody(String.class));
         mockCsv.assertIsSatisfied();
     }
 }


[2/2] camel git commit: Regen docs

Posted by da...@apache.org.
Regen docs


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

Branch: refs/heads/master
Commit: 4d6f2a901e5ab0312995be5b22a1703cb1301a6c
Parents: 0366bc1
Author: Claus Ibsen <da...@apache.org>
Authored: Wed Mar 1 09:34:15 2017 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Wed Mar 1 09:34:15 2017 +0100

----------------------------------------------------------------------
 camel-core/readme.adoc                                           | 2 +-
 components/camel-chunk/src/main/docs/chunk-component.adoc        | 4 ++--
 .../camel-freemarker/src/main/docs/freemarker-component.adoc     | 4 ++--
 components/camel-jolt/src/main/docs/jolt-component.adoc          | 4 ++--
 components/camel-mustache/src/main/docs/mustache-component.adoc  | 4 ++--
 components/camel-mvel/src/main/docs/mvel-component.adoc          | 4 ++--
 .../src/main/docs/string-template-component.adoc                 | 4 ++--
 components/camel-velocity/src/main/docs/velocity-component.adoc  | 4 ++--
 components/readme.adoc                                           | 2 +-
 9 files changed, 16 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/camel-core/readme.adoc
----------------------------------------------------------------------
diff --git a/camel-core/readme.adoc b/camel-core/readme.adoc
index ed2ca6a..c95b0c0 100644
--- a/camel-core/readme.adoc
+++ b/camel-core/readme.adoc
@@ -16,7 +16,7 @@ Number of Components: 25
 `bean:beanName` | 1.0 | The bean component is for invoking Java beans from Camel.
 
 | link:src/main/docs/binding-component.adoc[Binding] (camel-core) +
-`binding:bindingName:delegateUri` | 2.11 | The binding component is used for as a of wrapping an Endpoint in a contract with a data format.
+`binding:bindingName:delegateUri` | 2.11 | *deprecated* The binding component is used for as a of wrapping an Endpoint in a contract with a data format.
 
 | link:src/main/docs/browse-component.adoc[Browse] (camel-core) +
 `browse:name` | 1.3 | The browse component is used for viewing the messages received on endpoints that supports BrowsableEndpoint.

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-chunk/src/main/docs/chunk-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-chunk/src/main/docs/chunk-component.adoc b/components/camel-chunk/src/main/docs/chunk-component.adoc
index 03ecd9b..9211e62 100644
--- a/components/camel-chunk/src/main/docs/chunk-component.adoc
+++ b/components/camel-chunk/src/main/docs/chunk-component.adoc
@@ -48,7 +48,7 @@ The Chunk component supports 8 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | encoding | producer |  | String | Define the encoding of the body
 | extension | producer |  | String | Define the file extension of the template
@@ -173,4 +173,4 @@ Regards Camel Riders Bookstore
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-freemarker/src/main/docs/freemarker-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-freemarker/src/main/docs/freemarker-component.adoc b/components/camel-freemarker/src/main/docs/freemarker-component.adoc
index 709874c..c10f96c 100644
--- a/components/camel-freemarker/src/main/docs/freemarker-component.adoc
+++ b/components/camel-freemarker/src/main/docs/freemarker-component.adoc
@@ -63,7 +63,7 @@ The Freemarker component supports 6 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | configuration | producer |  | Configuration | Sets the Freemarker configuration to use
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | encoding | producer |  | String | Sets the encoding to be used for loading the template file.
@@ -226,4 +226,4 @@ And the java code:
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-jolt/src/main/docs/jolt-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-jolt/src/main/docs/jolt-component.adoc b/components/camel-jolt/src/main/docs/jolt-component.adoc
index a5efd01..f05370c 100644
--- a/components/camel-jolt/src/main/docs/jolt-component.adoc
+++ b/components/camel-jolt/src/main/docs/jolt-component.adoc
@@ -66,7 +66,7 @@ The JOLT component supports 6 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | inputType | producer | Hydrated | JoltInputOutputType | Specifies if the input is hydrated JSON or a JSON String.
 | outputType | producer | Hydrated | JoltInputOutputType | Specifies if the output should be hydrated JSON or a JSON String.
@@ -114,4 +114,4 @@ from("direct:in").
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-mustache/src/main/docs/mustache-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-mustache/src/main/docs/mustache-component.adoc b/components/camel-mustache/src/main/docs/mustache-component.adoc
index 2920b2f..cdec891 100644
--- a/components/camel-mustache/src/main/docs/mustache-component.adoc
+++ b/components/camel-mustache/src/main/docs/mustache-component.adoc
@@ -67,7 +67,7 @@ The Mustache component supports 6 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | encoding | producer |  | String | Character encoding of the resource content.
 | endDelimiter | producer | }} | String | Characters used to mark template code end.
@@ -174,4 +174,4 @@ Regards Camel Riders Bookstore
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-mvel/src/main/docs/mvel-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-mvel/src/main/docs/mvel-component.adoc b/components/camel-mvel/src/main/docs/mvel-component.adoc
index abd2e5e..54a78e2 100644
--- a/components/camel-mvel/src/main/docs/mvel-component.adoc
+++ b/components/camel-mvel/src/main/docs/mvel-component.adoc
@@ -50,7 +50,7 @@ The MVEL component supports 4 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | encoding | producer |  | String | Character encoding of the resource content.
 | synchronous | advanced | false | boolean | Sets whether synchronous processing should be strictly used or Camel is allowed to use asynchronous processing (if supported).
@@ -162,4 +162,4 @@ from("direct:in").
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-stringtemplate/src/main/docs/string-template-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-stringtemplate/src/main/docs/string-template-component.adoc b/components/camel-stringtemplate/src/main/docs/string-template-component.adoc
index 11e9c0d..a845ab0 100644
--- a/components/camel-stringtemplate/src/main/docs/string-template-component.adoc
+++ b/components/camel-stringtemplate/src/main/docs/string-template-component.adoc
@@ -49,7 +49,7 @@ The String Template component supports 5 endpoint options which are listed below
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | delimiterStart | producer | < | char | The variable start delimiter
 | delimiterStop | producer | > | char | The variable end delimiter
@@ -124,4 +124,4 @@ And the java code is as follows:
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/camel-velocity/src/main/docs/velocity-component.adoc
----------------------------------------------------------------------
diff --git a/components/camel-velocity/src/main/docs/velocity-component.adoc b/components/camel-velocity/src/main/docs/velocity-component.adoc
index 9207ca7..c473281 100644
--- a/components/camel-velocity/src/main/docs/velocity-component.adoc
+++ b/components/camel-velocity/src/main/docs/velocity-component.adoc
@@ -62,7 +62,7 @@ The Velocity component supports 6 endpoint options which are listed below:
 [width="100%",cols="2,1,1m,1m,5",options="header"]
 |=======================================================================
 | Name | Group | Default | Java Type | Description
-| resourceUri | producer |  | String | *Required* Path to the resource or a reference to lookup a bean in the Registry to use as the resource
+| resourceUri | producer |  | String | *Required* Path to the resource. You can prefix with: classpath file http ref or bean. classpath file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot eg bean:myBean.myMethod.
 | contentCache | producer | false | boolean | Sets whether to use resource content cache or not
 | encoding | producer |  | String | Character encoding of the resource content.
 | loaderCache | producer | true | boolean | Enables / disables the velocity resource loader cache which is enabled by default
@@ -256,4 +256,4 @@ And the java code:
 * link:configuring-camel.html[Configuring Camel]
 * link:component.html[Component]
 * link:endpoint.html[Endpoint]
-* link:getting-started.html[Getting Started]
\ No newline at end of file
+* link:getting-started.html[Getting Started]

http://git-wip-us.apache.org/repos/asf/camel/blob/4d6f2a90/components/readme.adoc
----------------------------------------------------------------------
diff --git a/components/readme.adoc b/components/readme.adoc
index 023fbb1..1d442f7 100644
--- a/components/readme.adoc
+++ b/components/readme.adoc
@@ -90,7 +90,7 @@ Number of Components: 220
 `beanstalk:connectionSettings` | 2.15 | The beanstalk component is used for job retrieval and post-processing of Beanstalk jobs.
 
 | link:../camel-core/src/main/docs/binding-component.adoc[Binding] (camel-core) +
-`binding:bindingName:delegateUri` | 2.11 | The binding component is used for as a of wrapping an Endpoint in a contract with a data format.
+`binding:bindingName:delegateUri` | 2.11 | *deprecated* The binding component is used for as a of wrapping an Endpoint in a contract with a data format.
 
 | link:camel-bonita/src/main/docs/bonita-component.adoc[Bonita] (camel-bonita) +
 `bonita:operation` | 2.19 | Camel Bonita support