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 2015/03/02 18:53:34 UTC

camel git commit: CAMEL-8332: Additional features for camel-dozer component

Repository: camel
Updated Branches:
  refs/heads/master 73b69af70 -> b23c7aa6e


CAMEL-8332: Additional features for camel-dozer component

This commit includes the following changes:
* Added a new converter which allows the user to evaluate an expression using Camel language support and assign it to an output field.
* Enabled javax.el support to allow variables to be used in mappings.
* Users can specify the name of a DozerBeanMapperConfiguration bean instead of a Dozer mapping file to allow for fine-grained configuration of the Dozer environment.
* The sourceModel option is actually used now and targetModel is marked as required.


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

Branch: refs/heads/master
Commit: b23c7aa6ed2ec313e789190e8d233a548e30c75a
Parents: 73b69af
Author: Keith Babo <kb...@redhat.com>
Authored: Sun Mar 1 21:53:44 2015 -0500
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Mar 2 18:53:45 2015 +0100

----------------------------------------------------------------------
 components/camel-dozer/pom.xml                  | 11 ++-
 .../camel/component/dozer/DozerComponent.java   | 30 ++++++-
 .../component/dozer/DozerConfiguration.java     | 13 ++-
 .../camel/component/dozer/DozerEndpoint.java    | 46 ++++++----
 .../camel/component/dozer/DozerProducer.java    | 24 +++++-
 .../camel/component/dozer/ExpressionMapper.java | 90 ++++++++++++++++++++
 .../camel/component/dozer/LiteralMapper.java    | 44 ----------
 .../camel/component/dozer/VariableMapper.java   | 44 ++++++++++
 .../dozer/DozerTypeConverterLoader.java         |  2 +-
 .../component/dozer/DozerComponentTest.java     | 12 +++
 .../component/dozer/ExpressionMappingTest.java  | 65 ++++++++++++++
 .../component/dozer/LiteralMappingTest.java     | 67 ---------------
 .../component/dozer/VariableMappingTest.java    | 67 +++++++++++++++
 .../dozer/CustomMappingTest-context.xml         | 24 ++++--
 .../dozer/ExpressionMappingTest-context.xml     | 29 +++++++
 .../dozer/LiteralMappingTest-context.xml        | 29 -------
 .../dozer/VariableMappingTest-context.xml       | 29 +++++++
 .../component/dozer/XmlToJsonTest-context.xml   |  2 +-
 .../camel/component/dozer/expressionMapping.xml | 39 +++++++++
 .../camel/component/dozer/literalMapping.xml    | 59 -------------
 .../camel/component/dozer/variableMapping.xml   | 65 ++++++++++++++
 .../features/src/main/resources/features.xml    |  2 +
 22 files changed, 562 insertions(+), 231 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/pom.xml b/components/camel-dozer/pom.xml
index e0f78bc..2bb8ac5 100644
--- a/components/camel-dozer/pom.xml
+++ b/components/camel-dozer/pom.xml
@@ -45,7 +45,16 @@
             <groupId>net.sf.dozer</groupId>
             <artifactId>dozer</artifactId>
         </dependency>
-
+        <dependency>
+          <groupId>javax.el</groupId>
+          <artifactId>javax.el-api</artifactId>
+          <version>${javax.el-api-version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.glassfish.web</groupId>
+          <artifactId>javax.el</artifactId>
+          <version>${javax.el-version}</version>
+        </dependency>
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerComponent.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerComponent.java
index dcae3ac..764bf35 100644
--- a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerComponent.java
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerComponent.java
@@ -16,26 +16,54 @@
  */
 package org.apache.camel.component.dozer;
 
+import java.lang.reflect.Field;
 import java.util.Map;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
+import org.apache.camel.converter.dozer.DozerBeanMapperConfiguration;
 import org.apache.camel.impl.UriEndpointComponent;
+import org.apache.camel.util.ReflectionHelper;
+import org.dozer.config.GlobalSettings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class DozerComponent extends UriEndpointComponent {
+    
+    private static final Logger LOG = LoggerFactory.getLogger(DozerComponent.class);
 
     public DozerComponent() {
         super(DozerEndpoint.class);
+        initDozerSettings();
     }
     
     public DozerComponent(CamelContext context) {
         super(context, DozerEndpoint.class);
+        initDozerSettings();
     }
-
+    
     protected Endpoint createEndpoint(String uri, String remaining, Map<String, Object> parameters) throws Exception {
         DozerConfiguration config = new DozerConfiguration();
         config.setName(remaining);
+        config.setMappingConfiguration(getAndRemoveOrResolveReferenceParameter(
+                parameters, "mappingConfiguration", DozerBeanMapperConfiguration.class));
         setProperties(config, parameters);
+        
+        // Validate endpoint parameters
+        if (config.getTargetModel() == null) {
+            throw new IllegalArgumentException("The targetModel parameter is required for dozer endpoints");
+        }
         return new DozerEndpoint(uri, this, config);
     }
+    
+    private void initDozerSettings() {
+        try {
+            GlobalSettings settings = GlobalSettings.getInstance();
+            LOG.info("Configuring GlobalSettings to enable EL");
+            Field field = settings.getClass().getDeclaredField("elEnabled");
+            ReflectionHelper.setField(field, settings, true);
+        } catch (NoSuchFieldException nsfEx) {
+            throw new IllegalStateException("Failed to enable EL in global Dozer settings", nsfEx);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerConfiguration.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerConfiguration.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerConfiguration.java
index 94f9e3d..edae400 100644
--- a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerConfiguration.java
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerConfiguration.java
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.component.dozer;
 
+import org.apache.camel.converter.dozer.DozerBeanMapperConfiguration;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.UriParam;
 import org.apache.camel.spi.UriParams;
@@ -37,10 +38,12 @@ public class DozerConfiguration {
     private String unmarshalId;
     @UriParam
     private String sourceModel;
-    @UriParam
+    @UriParam @Metadata(required = "true")
     private String targetModel;
     @UriParam(defaultValue = DEFAULT_MAPPING_FILE)
     private String mappingFile;
+    @UriParam
+    private DozerBeanMapperConfiguration mappingConfiguration;
     
     public DozerConfiguration() {
         setMappingFile(DEFAULT_MAPPING_FILE);
@@ -93,4 +96,12 @@ public class DozerConfiguration {
     public void setMappingFile(String mappingFile) {
         this.mappingFile = mappingFile;
     }
+    
+    public DozerBeanMapperConfiguration getMappingConfiguration() {
+        return mappingConfiguration;
+    }
+
+    public void setMappingConfiguration(DozerBeanMapperConfiguration mappingConfiguration) {
+        this.mappingConfiguration = mappingConfiguration;
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerEndpoint.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerEndpoint.java
index 292a21a..f4a40ca 100644
--- a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerEndpoint.java
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerEndpoint.java
@@ -24,6 +24,7 @@ import org.apache.camel.Component;
 import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
+import org.apache.camel.converter.dozer.DozerTypeConverterLoader;
 import org.apache.camel.impl.DefaultEndpoint;
 import org.apache.camel.spi.UriEndpoint;
 import org.apache.camel.spi.UriParam;
@@ -40,11 +41,13 @@ public class DozerEndpoint extends DefaultEndpoint {
 
     // IDs for built-in custom converters used with the Dozer component
     private static final String CUSTOM_MAPPING_ID = "_customMapping";
-    private static final String LITERAL_MAPPING_ID = "_literalMapping";
+    private static final String VARIABLE_MAPPING_ID = "_variableMapping";
+    private static final String EXPRESSION_MAPPING_ID = "_expressionMapping";
 
     private DozerBeanMapper mapper;
-    private LiteralMapper literalMapper;
+    private VariableMapper variableMapper;
     private CustomMapper customMapper;
+    private ExpressionMapper expressionMapper;
 
     @UriParam
     private DozerConfiguration configuration;
@@ -52,8 +55,9 @@ public class DozerEndpoint extends DefaultEndpoint {
     public DozerEndpoint(String endpointUri, Component component, DozerConfiguration configuration) throws Exception {
         super(endpointUri, component);
         this.configuration = configuration;
-        literalMapper = new LiteralMapper();
+        variableMapper = new VariableMapper();
         customMapper = new CustomMapper(getCamelContext().getClassResolver());
+        expressionMapper = new ExpressionMapper();
     }
 
     @Override
@@ -87,8 +91,12 @@ public class DozerEndpoint extends DefaultEndpoint {
         return customMapper;
     }
     
-    LiteralMapper getLiteralMapper() {
-        return literalMapper;
+    VariableMapper getVariableMapper() {
+        return variableMapper;
+    }
+    
+    ExpressionMapper getExpressionMapper() {
+        return expressionMapper;
     }
 
     @Override
@@ -96,7 +104,13 @@ public class DozerEndpoint extends DefaultEndpoint {
         super.doStart();
 
         if (mapper == null) {
-            mapper = createDozerBeanMapper();
+            if (configuration.getMappingConfiguration() != null) {
+                mapper = DozerTypeConverterLoader.createDozerBeanMapper(
+                        configuration.getMappingConfiguration());
+            } else {
+                mapper = createDozerBeanMapper();
+            }
+            configureMapper(mapper);
         }
     }
 
@@ -105,7 +119,7 @@ public class DozerEndpoint extends DefaultEndpoint {
         super.doStop();
         // noop
     }
-
+    
     private DozerBeanMapper createDozerBeanMapper() throws Exception {
         DozerBeanMapper answer = new DozerBeanMapper();
         InputStream mapStream = null;
@@ -114,18 +128,20 @@ public class DozerEndpoint extends DefaultEndpoint {
             // create the mapper instance and add the mapping file
             mapStream = ResourceHelper.resolveMandatoryResourceAsInputStream(getCamelContext().getClassResolver(), configuration.getMappingFile());
             answer.addMapping(mapStream);
-            
-            // add our built-in converters
-            Map<String, CustomConverter> converters = new HashMap<String, CustomConverter>();
-            converters.put(CUSTOM_MAPPING_ID, customMapper);
-            converters.put(LITERAL_MAPPING_ID, literalMapper);
-            converters.putAll(answer.getCustomConvertersWithId());
-            answer.setCustomConvertersWithId(converters);
-            
         } finally {
             IOHelper.close(mapStream);
         }
 
         return answer;
     }
+
+    private void configureMapper(DozerBeanMapper mapper) throws Exception {
+        // add our built-in converters
+        Map<String, CustomConverter> converters = new HashMap<String, CustomConverter>();
+        converters.put(CUSTOM_MAPPING_ID, customMapper);
+        converters.put(VARIABLE_MAPPING_ID, variableMapper);
+        converters.put(EXPRESSION_MAPPING_ID, expressionMapper);
+        converters.putAll(mapper.getCustomConvertersWithId());
+        mapper.setCustomConvertersWithId(converters);
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerProducer.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerProducer.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerProducer.java
index 1de9cf3..4cd1159 100644
--- a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerProducer.java
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/DozerProducer.java
@@ -58,14 +58,30 @@ public class DozerProducer extends DefaultProducer {
         // Load the target model class
         Class<?> targetModel = endpoint.getCamelContext().getClassResolver().resolveClass(
                 endpoint.getConfiguration().getTargetModel());
-        // If an unmarshaller was used, the unmarshalled message is the OUT
-        // message.
+        
+        // If an unmarshaller was used, the unmarshalled message is the OUT message.
         Message msg = exchange.hasOut() ? exchange.getOut() : exchange.getIn();
+        
+        // Convert to source model, if specified
+        String sourceType = endpoint.getConfiguration().getSourceModel();
+        if (sourceType != null) {
+            LOG.debug("Converting to source model {}.", sourceType);
+            Class<?> sourceModel = endpoint.getCamelContext()
+                    .getClassResolver().resolveClass(sourceType);
+            if (sourceModel == null) {
+                throw new Exception("Unable to load sourceModel class: " + sourceType);
+            }
+            msg.setBody(msg.getBody(sourceModel));
+        }
+        
+        // Perform mappings
         LOG.debug("Mapping to target model {}.", targetModel.getName());
-        // Trigger the Dozer mapping and set that as the content of the IN message
         Object targetObject = endpoint.getMapper().map(msg.getBody(), targetModel);
         // Second pass to process literal mappings
-        endpoint.getMapper().map(endpoint.getLiteralMapper(), targetObject);
+        endpoint.getMapper().map(endpoint.getVariableMapper(), targetObject);
+        // Third pass to process expression mappings
+        endpoint.getExpressionMapper().setCurrentExchange(exchange);
+        endpoint.getMapper().map(endpoint.getExpressionMapper(), targetObject);
         msg.setBody(targetObject);
         exchange.setIn(msg);
         

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/ExpressionMapper.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/ExpressionMapper.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/ExpressionMapper.java
new file mode 100644
index 0000000..cf9664d
--- /dev/null
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/ExpressionMapper.java
@@ -0,0 +1,90 @@
+/**
+ * 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.component.dozer;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Expression;
+import org.apache.camel.spi.Language;
+
+/**
+ * Provides support for mapping a Camel expression to a target field in a 
+ * mapping.  Expressions have the following format:
+ * <br><br>
+ * [language]:[expression]
+ * <br><br>
+ */
+public class ExpressionMapper extends BaseConverter {
+    
+    private ThreadLocal<Exchange> currentExchange = new ThreadLocal<Exchange>();
+    
+    @Override
+    public Object convert(Object existingDestinationFieldValue, 
+            Object sourceFieldValue, 
+            Class<?> destinationClass,
+            Class<?> sourceClass) {
+        try {
+            if (currentExchange.get() == null) {
+                throw new IllegalStateException(
+                        "Current exchange has not been set for ExpressionMapper");
+            }
+            // Resolve the language being used for this expression and evaluate
+            Exchange exchange = currentExchange.get();
+            Language expLang = exchange.getContext().resolveLanguage(getLanguagePart());
+            Expression exp = expLang.createExpression(getExpressionPart());
+            return exp.evaluate(exchange, destinationClass);
+        } finally {
+            done();
+        }
+    }
+    
+    /**
+     * Used as the source field for Dozer mappings. 
+     */
+    public String getExpression() {
+        return getParameter();
+    }
+    
+    /**
+     * The actual expression, without the language prefix.
+     */
+    public String getExpressionPart() {
+        return getParameter().substring(getParameter().indexOf(":") + 1);
+    }
+    
+    /**
+     * The expression language used for this mapping.
+     */
+    public String getLanguagePart() {
+        return getParameter().substring(0, getParameter().indexOf(":"));
+    }
+    
+    /**
+     * Sets the Camel exchange reference for this mapping.  The exchange 
+     * reference is stored in a thread-local which is cleaned up after the 
+     * mapping has been performed via the done() method.
+     * @param exchange
+     */
+    public void setCurrentExchange(Exchange exchange) {
+        currentExchange.set(exchange);
+    }
+    
+    @Override
+    public void done() {
+        super.done();
+        currentExchange.set(null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/LiteralMapper.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/LiteralMapper.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/LiteralMapper.java
deleted file mode 100644
index 4d09963..0000000
--- a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/LiteralMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.component.dozer;
-
-/**
- * Used to map literal values (e.g. "ACME" or "ABC-123") to a field in the 
- * target object.
- */
-public class LiteralMapper extends BaseConverter {
-     
-    @Override
-    public Object convert(Object existingDestinationFieldValue, 
-            Object sourceFieldValue, 
-            Class<?> destinationClass,
-            Class<?> sourceClass) {
-        try {
-            return getParameter();
-        } finally {
-            done();
-        }
-    }
-    
-    /**
-     * We need at least one field in this class so that we can use it as a
-     * source for Dozer transformations.
-     */
-    public String getLiteral() {
-        return getParameter();
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/VariableMapper.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/VariableMapper.java b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/VariableMapper.java
new file mode 100644
index 0000000..ac0b2f5
--- /dev/null
+++ b/components/camel-dozer/src/main/java/org/apache/camel/component/dozer/VariableMapper.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.component.dozer;
+
+/**
+ * Used to map literal values (e.g. "ACME" or "ABC-123") to a field in the 
+ * target object.
+ */
+public class VariableMapper extends BaseConverter {
+     
+    @Override
+    public Object convert(Object existingDestinationFieldValue, 
+            Object sourceFieldValue, 
+            Class<?> destinationClass,
+            Class<?> sourceClass) {
+        try {
+            return getParameter();
+        } finally {
+            done();
+        }
+    }
+    
+    /**
+     * We need at least one field in this class so that we can use it as a
+     * source for Dozer transformations.
+     */
+    public String getLiteral() {
+        return getParameter();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/main/java/org/apache/camel/converter/dozer/DozerTypeConverterLoader.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/main/java/org/apache/camel/converter/dozer/DozerTypeConverterLoader.java b/components/camel-dozer/src/main/java/org/apache/camel/converter/dozer/DozerTypeConverterLoader.java
index 6118caf..588f88e 100644
--- a/components/camel-dozer/src/main/java/org/apache/camel/converter/dozer/DozerTypeConverterLoader.java
+++ b/components/camel-dozer/src/main/java/org/apache/camel/converter/dozer/DozerTypeConverterLoader.java
@@ -228,7 +228,7 @@ public class DozerTypeConverterLoader extends ServiceSupport implements CamelCon
      * @param configuration  the dozer bean mapper configuration.
      * @return the created mapper
      */
-    protected DozerBeanMapper createDozerBeanMapper(DozerBeanMapperConfiguration configuration) {
+    public static DozerBeanMapper createDozerBeanMapper(DozerBeanMapperConfiguration configuration) {
         DozerBeanMapper mapper;
         if (configuration.getMappingFiles() != null) {
             mapper = new DozerBeanMapper(configuration.getMappingFiles());

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/DozerComponentTest.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/DozerComponentTest.java b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/DozerComponentTest.java
index 4dad0f0..7cc9751 100644
--- a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/DozerComponentTest.java
+++ b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/DozerComponentTest.java
@@ -49,4 +49,16 @@ public class DozerComponentTest {
         Assert.assertEquals(TARGET_MODEL, config.getTargetModel());
         Assert.assertEquals(DOZER_CONFIG_PATH, config.getMappingFile());
     }
+    
+    @Test
+    public void requiredTargetModelMissing() throws Exception {
+        DozerComponent comp = new DozerComponent();
+        comp.setCamelContext(new DefaultCamelContext());
+        try {
+            comp.createEndpoint("dozer:noTargetModel?mappingFile=mapping.xml");
+            Assert.fail("targetModel is a required parameter");
+        } catch (IllegalArgumentException ex) {
+            // expected
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/ExpressionMappingTest.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/ExpressionMappingTest.java b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/ExpressionMappingTest.java
new file mode 100644
index 0000000..97d6bf7
--- /dev/null
+++ b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/ExpressionMappingTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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.component.dozer;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.EndpointInject;
+import org.apache.camel.Produce;
+import org.apache.camel.ProducerTemplate;
+import org.apache.camel.component.dozer.example.abc.ABCOrder;
+import org.apache.camel.component.dozer.example.xyz.XYZOrder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.test.context.ContextConfiguration;
+import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
+
+@RunWith(SpringJUnit4ClassRunner.class)
+@ContextConfiguration
+public class ExpressionMappingTest {
+    
+    @EndpointInject(uri = "mock:result")
+    private MockEndpoint resultEndpoint;
+    
+    @Produce(uri = "direct:start")
+    private ProducerTemplate startEndpoint;
+    
+    @Autowired
+    private CamelContext camelContext;
+    
+    @After
+    public void tearDown() {
+        resultEndpoint.reset();
+    }
+    
+    @Test
+    public void testExpressionMapping() throws Exception {
+        resultEndpoint.expectedMessageCount(1);
+        final String headerName = "customerNumber";
+        final String headerVal = "CAFE-123";
+        ABCOrder abcOrder = new ABCOrder();
+        // Header value should be mapped to custId in target model
+        startEndpoint.sendBodyAndHeader(abcOrder, headerName, headerVal);
+        // check results
+        resultEndpoint.assertIsSatisfied();
+        XYZOrder result = resultEndpoint.getExchanges().get(0).getIn().getBody(XYZOrder.class);
+        Assert.assertEquals(headerVal, result.getCustId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/LiteralMappingTest.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/LiteralMappingTest.java b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/LiteralMappingTest.java
deleted file mode 100644
index f519991..0000000
--- a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/LiteralMappingTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * 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.component.dozer;
-
-import org.apache.camel.CamelContext;
-import org.apache.camel.EndpointInject;
-import org.apache.camel.Produce;
-import org.apache.camel.ProducerTemplate;
-import org.apache.camel.component.dozer.example.abc.ABCOrder;
-import org.apache.camel.component.dozer.example.abc.ABCOrder.Header;
-import org.apache.camel.component.dozer.example.xyz.XYZOrder;
-import org.apache.camel.component.mock.MockEndpoint;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.test.context.ContextConfiguration;
-import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
-
-@RunWith(SpringJUnit4ClassRunner.class)
-@ContextConfiguration
-public class LiteralMappingTest {
-    
-    @EndpointInject(uri = "mock:result")
-    private MockEndpoint resultEndpoint;
-    
-    @Produce(uri = "direct:start")
-    private ProducerTemplate startEndpoint;
-    
-    @Autowired
-    private CamelContext camelContext;
-    
-    @After
-    public void tearDown() {
-        resultEndpoint.reset();
-    }
-    
-    @Test
-    public void testLiteralMapping() throws Exception {
-        resultEndpoint.expectedMessageCount(1);
-        ABCOrder abcOrder = new ABCOrder();
-        abcOrder.setHeader(new Header());
-        abcOrder.getHeader().setStatus("GOLD");
-        startEndpoint.sendBody(abcOrder);
-        // check results
-        resultEndpoint.assertIsSatisfied();
-        XYZOrder result = resultEndpoint.getExchanges().get(0).getIn().getBody(XYZOrder.class);
-        Assert.assertEquals(result.getPriority(), "GOLD");
-        Assert.assertEquals(result.getCustId(), "LITERAL_CUST_ID");
-        Assert.assertEquals(result.getOrderId(), "LITERAL_ORDER_ID");
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/VariableMappingTest.java
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/VariableMappingTest.java b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/VariableMappingTest.java
new file mode 100644
index 0000000..61de4a9
--- /dev/null
+++ b/components/camel-dozer/src/test/java/org/apache/camel/component/dozer/VariableMappingTest.java
@@ -0,0 +1,67 @@
+/**
+ * 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.component.dozer;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.EndpointInject;
+import org.apache.camel.Produce;
+import org.apache.camel.ProducerTemplate;
+import org.apache.camel.component.dozer.example.abc.ABCOrder;
+import org.apache.camel.component.dozer.example.abc.ABCOrder.Header;
+import org.apache.camel.component.dozer.example.xyz.XYZOrder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.test.context.ContextConfiguration;
+import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
+
+@RunWith(SpringJUnit4ClassRunner.class)
+@ContextConfiguration
+public class VariableMappingTest {
+    
+    @EndpointInject(uri = "mock:result")
+    private MockEndpoint resultEndpoint;
+    
+    @Produce(uri = "direct:start")
+    private ProducerTemplate startEndpoint;
+    
+    @Autowired
+    private CamelContext camelContext;
+    
+    @After
+    public void tearDown() {
+        resultEndpoint.reset();
+    }
+    
+    @Test
+    public void testLiteralMapping() throws Exception {
+        resultEndpoint.expectedMessageCount(1);
+        ABCOrder abcOrder = new ABCOrder();
+        abcOrder.setHeader(new Header());
+        abcOrder.getHeader().setStatus("GOLD");
+        startEndpoint.sendBody(abcOrder);
+        // check results
+        resultEndpoint.assertIsSatisfied();
+        XYZOrder result = resultEndpoint.getExchanges().get(0).getIn().getBody(XYZOrder.class);
+        Assert.assertEquals(result.getPriority(), "GOLD");
+        Assert.assertEquals("ACME-SALES", result.getCustId());
+        Assert.assertEquals("W123-EG", result.getOrderId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/CustomMappingTest-context.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/CustomMappingTest-context.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/CustomMappingTest-context.xml
index fde0332..8ca1449 100644
--- a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/CustomMappingTest-context.xml
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/CustomMappingTest-context.xml
@@ -18,12 +18,20 @@
 <beans xmlns="http://www.springframework.org/schema/beans" xmlns:camel="http://camel.apache.org/schema/spring" 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">
 
    <!-- Camel route -->
-   <camelContext xmlns="http://camel.apache.org/schema/spring">
-    <endpoint uri="dozer:java2java?mappingFile=org/apache/camel/component/dozer/customMapping.xml&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder" id="java2java"/>
-    <route>
-        <from uri="direct:start"/>
-        <to ref="java2java"/>
-        <to uri="mock:result"/>
-    </route>
-</camelContext>
+    <camelContext xmlns="http://camel.apache.org/schema/spring">
+	  <endpoint uri="dozer:java2java?mappingConfiguration=#mapConfig&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder" id="java2java"/>
+	  <route>
+	      <from uri="direct:start"/>
+	      <to ref="java2java"/>
+	      <to uri="mock:result"/>
+	  </route>
+    </camelContext>
+    
+    <bean id="mapConfig" class="org.apache.camel.converter.dozer.DozerBeanMapperConfiguration">
+      <property name="mappingFiles">
+        <list>
+          <value>org/apache/camel/component/dozer/customMapping.xml</value>
+        </list>
+      </property>
+    </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/ExpressionMappingTest-context.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/ExpressionMappingTest-context.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/ExpressionMappingTest-context.xml
new file mode 100644
index 0000000..8bc7212
--- /dev/null
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/ExpressionMappingTest-context.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  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:camel="http://camel.apache.org/schema/spring" 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">
+
+   <!-- Camel route -->
+   <camelContext xmlns="http://camel.apache.org/schema/spring">
+    <endpoint uri="dozer:java2java?mappingFile=org/apache/camel/component/dozer/expressionMapping.xml&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder&amp;sourceModel=org.apache.camel.component.dozer.example.abc.ABCOrder" id="java2java"/>
+    <route>
+        <from uri="direct:start"/>
+        <to ref="java2java"/>
+        <to uri="mock:result"/>
+    </route>
+</camelContext>
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/LiteralMappingTest-context.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/LiteralMappingTest-context.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/LiteralMappingTest-context.xml
deleted file mode 100644
index f13d5a9..0000000
--- a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/LiteralMappingTest-context.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
-  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:camel="http://camel.apache.org/schema/spring" 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">
-
-   <!-- Camel route -->
-   <camelContext xmlns="http://camel.apache.org/schema/spring">
-    <endpoint uri="dozer:java2java?mappingFile=org/apache/camel/component/dozer/literalMapping.xml&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder" id="java2java"/>
-    <route>
-        <from uri="direct:start"/>
-        <to ref="java2java"/>
-        <to uri="mock:result"/>
-    </route>
-</camelContext>
-</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/VariableMappingTest-context.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/VariableMappingTest-context.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/VariableMappingTest-context.xml
new file mode 100644
index 0000000..a3bef03
--- /dev/null
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/VariableMappingTest-context.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  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:camel="http://camel.apache.org/schema/spring" 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">
+
+   <!-- Camel route -->
+   <camelContext xmlns="http://camel.apache.org/schema/spring">
+    <endpoint uri="dozer:java2java?mappingFile=org/apache/camel/component/dozer/variableMapping.xml&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder&amp;sourceModel=org.apache.camel.component.dozer.example.abc.ABCOrder" id="java2java"/>
+    <route>
+        <from uri="direct:start"/>
+        <to ref="java2java"/>
+        <to uri="mock:result"/>
+    </route>
+</camelContext>
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/XmlToJsonTest-context.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/XmlToJsonTest-context.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/XmlToJsonTest-context.xml
index e9678d0..bc984ba 100644
--- a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/XmlToJsonTest-context.xml
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/XmlToJsonTest-context.xml
@@ -19,7 +19,7 @@
 
    <!-- Camel route -->
    <camelContext xmlns="http://camel.apache.org/schema/spring">
-    <endpoint uri="dozer:xml2json?mappingFile=org/apache/camel/component/dozer/dozerBeanMapping.xml&amp;marshalId=myjson&amp;unmarshalId=myjaxb&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder" id="xml2json"/>
+    <endpoint uri="dozer:xml2json?mappingFile=org/apache/camel/component/dozer/dozerBeanMapping.xml&amp;marshalId=myjson&amp;unmarshalId=myjaxb&amp;targetModel=org.apache.camel.component.dozer.example.xyz.XYZOrder&amp;sourceModel=org.apache.camel.component.dozer.example.abc.ABCOrder" id="xml2json"/>
     <dataFormats>
         <json library="Jackson" id="myjson"/>
         <jaxb contextPath="org.apache.camel.component.dozer.example.abc" id="myjaxb"/>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/expressionMapping.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/expressionMapping.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/expressionMapping.xml
new file mode 100644
index 0000000..c4aa084
--- /dev/null
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/expressionMapping.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<mappings xmlns="http://dozer.sourceforge.net" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://dozer.sourceforge.net http://dozer.sourceforge.net/schema/beanmapping.xsd">
+    <mapping>
+        <class-a>org.apache.camel.component.dozer.example.abc.ABCOrder</class-a>
+        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
+        <field>
+            <a>header.status</a>
+            <b>priority</b>
+        </field>
+        <field>
+            <a>header.orderNum</a>
+            <b>orderId</b>
+        </field>
+    </mapping>
+    <mapping>
+        <class-a>org.apache.camel.component.dozer.ExpressionMapper</class-a>
+        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
+        <field custom-converter-id="_expressionMapping" custom-converter-param="simple:\${header.customerNumber}">
+            <a>expression</a>
+            <b>custId</b>
+        </field>
+    </mapping>
+</mappings>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/literalMapping.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/literalMapping.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/literalMapping.xml
deleted file mode 100644
index 481f0a4..0000000
--- a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/literalMapping.xml
+++ /dev/null
@@ -1,59 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  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.
--->
-<mappings xmlns="http://dozer.sourceforge.net" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://dozer.sourceforge.net http://dozer.sourceforge.net/schema/beanmapping.xsd">
-    <mapping>
-        <class-a>org.apache.camel.component.dozer.example.abc.ABCOrder</class-a>
-        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
-        <field>
-            <a>header.status</a>
-            <b>priority</b>
-        </field>
-        <field>
-            <a>header.orderNum</a>
-            <b>orderId</b>
-        </field>
-    </mapping>
-    <mapping>
-        <class-a>org.apache.camel.component.dozer.LiteralMapper</class-a>
-        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
-        <field custom-converter-id="_literalMapping" custom-converter-param="LITERAL_CUST_ID">
-            <a>literal</a>
-            <b>custId</b>
-        </field>
-        <field custom-converter-id="_literalMapping" custom-converter-param="LITERAL_ORDER_ID">
-            <a>literal</a>
-            <b>orderId</b>
-        </field>
-    </mapping>
-    <mapping>
-        <class-a>org.apache.camel.component.dozer.example.abc.ABCOrder$OrderItems$Item</class-a>
-        <class-b>org.apache.camel.component.dozer.example.xyz.LineItem</class-b>
-        <field>
-            <a>id</a>
-            <b>itemId</b>
-        </field>
-        <field>
-            <a>price</a>
-            <b>cost</b>
-        </field>
-        <field>
-            <a>quantity</a>
-            <b>amount</b>
-        </field>
-    </mapping>
-</mappings>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/variableMapping.xml
----------------------------------------------------------------------
diff --git a/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/variableMapping.xml b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/variableMapping.xml
new file mode 100644
index 0000000..166d207
--- /dev/null
+++ b/components/camel-dozer/src/test/resources/org/apache/camel/component/dozer/variableMapping.xml
@@ -0,0 +1,65 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<mappings xmlns="http://dozer.sourceforge.net" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://dozer.sourceforge.net http://dozer.sourceforge.net/schema/beanmapping.xsd">
+    <configuration>
+        <variables>
+            <variable name="CUST_ID">ACME-SALES</variable>
+            <variable name="ORDER_ID">W123-EG</variable>
+        </variables>
+    </configuration>
+    <mapping>
+        <class-a>org.apache.camel.component.dozer.example.abc.ABCOrder</class-a>
+        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
+        <field>
+            <a>header.status</a>
+            <b>priority</b>
+        </field>
+        <field>
+            <a>header.orderNum</a>
+            <b>orderId</b>
+        </field>
+    </mapping>
+    <mapping>
+        <class-a>org.apache.camel.component.dozer.VariableMapper</class-a>
+        <class-b>org.apache.camel.component.dozer.example.xyz.XYZOrder</class-b>
+        <field custom-converter-id="_variableMapping" custom-converter-param="${CUST_ID}">
+            <a>literal</a>
+            <b>custId</b>
+        </field>
+        <field custom-converter-id="_variableMapping" custom-converter-param="${ORDER_ID}">
+            <a>literal</a>
+            <b>orderId</b>
+        </field>
+    </mapping>
+    <mapping>
+        <class-a>org.apache.camel.component.dozer.example.abc.ABCOrder$OrderItems$Item</class-a>
+        <class-b>org.apache.camel.component.dozer.example.xyz.LineItem</class-b>
+        <field>
+            <a>id</a>
+            <b>itemId</b>
+        </field>
+        <field>
+            <a>price</a>
+            <b>cost</b>
+        </field>
+        <field>
+            <a>quantity</a>
+            <b>amount</b>
+        </field>
+    </mapping>
+</mappings>

http://git-wip-us.apache.org/repos/asf/camel/blob/b23c7aa6/platforms/karaf/features/src/main/resources/features.xml
----------------------------------------------------------------------
diff --git a/platforms/karaf/features/src/main/resources/features.xml b/platforms/karaf/features/src/main/resources/features.xml
index 7bf0838..73a3e5f 100644
--- a/platforms/karaf/features/src/main/resources/features.xml
+++ b/platforms/karaf/features/src/main/resources/features.xml
@@ -316,6 +316,8 @@
     <bundle dependency='true'>mvn:org.apache.commons/commons-lang3/${commons-lang3-version}</bundle>
     <bundle dependency='true'>mvn:commons-collections/commons-collections/${commons-collections-version}</bundle>
     <bundle dependency='true'>mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.commons-beanutils/${commons-beanutils-bundle-version}</bundle>
+    <bundle dependency='true'>mvn:javax.el/javax.el-api/${javax.el-api-version}</bundle>
+    <bundle dependency='true'>mvn:org.glassfish.web/javax.el/${javax.el-version}</bundle>
     <bundle>mvn:org.apache.camel/camel-dozer/${project.version}</bundle>
   </feature>
   <feature name='camel-dropbox' version='${project.version}' resolver='(obr)' start-level='50'>