You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by gn...@apache.org on 2018/10/12 07:55:57 UTC

[camel] 43/43: Reifiers

This is an automated email from the ASF dual-hosted git repository.

gnodet pushed a commit to branch camel-3.x
in repository https://gitbox.apache.org/repos/asf/camel.git

commit 5533f060a46b2c8f75f7c5c9c26eca3fa9b4ff4a
Author: Guillaume Nodet <gn...@gmail.com>
AuthorDate: Wed Sep 5 12:59:09 2018 +0200

    Reifiers
---
 .../org/apache/camel/impl/DefaultCamelContext.java |   3 +-
 .../org/apache/camel/impl/DefaultRouteContext.java |   6 +-
 .../camel/impl/DefaultTransformerRegistry.java     |   3 +-
 .../camel/impl/DefaultValidatorRegistry.java       |   3 +-
 .../apache/camel/model/AggregateDefinition.java    | 171 ------
 .../org/apache/camel/model/BeanDefinition.java     | 131 +----
 .../org/apache/camel/model/CatchDefinition.java    |  48 --
 .../org/apache/camel/model/ChoiceDefinition.java   |  30 --
 .../apache/camel/model/ClaimCheckDefinition.java   |  92 ----
 .../apache/camel/model/ConvertBodyDefinition.java  |  30 --
 .../org/apache/camel/model/DelayDefinition.java    |  34 --
 .../camel/model/DynamicRouterDefinition.java       |  30 --
 .../org/apache/camel/model/EnrichDefinition.java   |  54 +-
 .../org/apache/camel/model/ExpressionNode.java     |  46 +-
 .../org/apache/camel/model/FilterDefinition.java   |  15 -
 .../org/apache/camel/model/FinallyDefinition.java  |  13 -
 .../org/apache/camel/model/HystrixDefinition.java  |   9 +-
 .../camel/model/IdempotentConsumerDefinition.java  |  37 --
 .../apache/camel/model/InterceptDefinition.java    |  48 +-
 .../camel/model/InterceptFromDefinition.java       |  27 -
 .../model/InterceptSendToEndpointDefinition.java   |  71 ---
 .../apache/camel/model/LoadBalanceDefinition.java  |  44 +-
 .../apache/camel/model/LoadBalancerDefinition.java |  31 +-
 .../java/org/apache/camel/model/LogDefinition.java |  73 ---
 .../org/apache/camel/model/LoopDefinition.java     |  19 -
 .../org/apache/camel/model/MarshalDefinition.java  |  10 -
 .../apache/camel/model/MulticastDefinition.java    |  83 ---
 .../apache/camel/model/OnCompletionDefinition.java |  62 +--
 .../apache/camel/model/OnExceptionDefinition.java  | 136 +----
 .../apache/camel/model/OnFallbackDefinition.java   |   7 -
 .../apache/camel/model/OtherwiseDefinition.java    |   7 -
 .../org/apache/camel/model/PipelineDefinition.java |   5 -
 .../org/apache/camel/model/PolicyDefinition.java   |  42 +-
 .../apache/camel/model/PollEnrichDefinition.java   |  58 ---
 .../org/apache/camel/model/ProcessDefinition.java  |  30 +-
 .../apache/camel/model/ProcessorDefinition.java    | 380 +-------------
 .../camel/model/RecipientListDefinition.java       | 119 -----
 .../apache/camel/model/RemoveHeaderDefinition.java |  10 -
 .../camel/model/RemoveHeadersDefinition.java       |  16 -
 .../camel/model/RemovePropertiesDefinition.java    |  16 -
 .../camel/model/RemovePropertyDefinition.java      |  10 -
 .../apache/camel/model/ResequenceDefinition.java   | 103 ----
 .../org/apache/camel/model/RethrowDefinition.java  |  10 -
 .../org/apache/camel/model/RollbackDefinition.java |  19 -
 .../org/apache/camel/model/RouteDefinition.java    | 283 +---------
 .../apache/camel/model/RoutingSlipDefinition.java  |  29 --
 .../org/apache/camel/model/SagaDefinition.java     |  78 ---
 .../org/apache/camel/model/SamplingDefinition.java |  18 -
 .../org/apache/camel/model/ScriptDefinition.java   |   9 -
 .../org/apache/camel/model/SendDefinition.java     |  18 -
 .../org/apache/camel/model/SetBodyDefinition.java  |   9 -
 .../camel/model/SetExchangePatternDefinition.java  |  16 +-
 .../apache/camel/model/SetFaultBodyDefinition.java |   9 -
 .../apache/camel/model/SetHeaderDefinition.java    |  12 -
 .../apache/camel/model/SetPropertyDefinition.java  |  12 -
 .../org/apache/camel/model/SortDefinition.java     |  29 --
 .../org/apache/camel/model/SplitDefinition.java    |  65 ---
 .../org/apache/camel/model/StopDefinition.java     |   8 -
 .../org/apache/camel/model/ThreadsDefinition.java  |  97 +---
 .../org/apache/camel/model/ThrottleDefinition.java |  48 --
 .../camel/model/ThrowExceptionDefinition.java      |  24 -
 .../apache/camel/model/ToDynamicDefinition.java    | 145 ------
 .../apache/camel/model/TransactedDefinition.java   | 127 +----
 .../apache/camel/model/TransformDefinition.java    |   9 -
 .../java/org/apache/camel/model/TryDefinition.java |  34 +-
 .../apache/camel/model/UnmarshalDefinition.java    |   9 -
 .../org/apache/camel/model/ValidateDefinition.java |   8 -
 .../org/apache/camel/model/WhenDefinition.java     |   7 -
 .../model/WhenSkipSendToEndpointDefinition.java    |  23 -
 .../org/apache/camel/model/WireTapDefinition.java  |  67 +--
 .../camel/model/cloud/ServiceCallDefinition.java   | 426 ---------------
 .../loadbalancer/CustomLoadBalancerDefinition.java |  21 +-
 .../FailoverLoadBalancerDefinition.java            |  42 --
 .../loadbalancer/RandomLoadBalancerDefinition.java |   7 -
 .../RoundRobinLoadBalancerDefinition.java          |   7 -
 .../loadbalancer/StickyLoadBalancerDefinition.java |   8 -
 .../loadbalancer/TopicLoadBalancerDefinition.java  |   7 -
 .../WeightedLoadBalancerDefinition.java            |  33 --
 .../camel/model/rest/RestBindingDefinition.java    | 182 +------
 .../transformer/CustomTransformerDefinition.java   |  29 --
 .../DataFormatTransformerDefinition.java           |  17 +-
 .../transformer/EndpointTransformerDefinition.java |  24 +-
 .../model/transformer/TransformerDefinition.java   |  12 +-
 .../model/validator/CustomValidatorDefinition.java |  27 -
 .../validator/EndpointValidatorDefinition.java     |  19 +-
 .../validator/PredicateValidatorDefinition.java    |  17 +-
 .../camel/model/validator/ValidatorDefinition.java |   7 -
 .../org/apache/camel/reifier/AggregateReifier.java | 208 ++++++++
 .../BeanReifier.java}                              | 151 +-----
 .../org/apache/camel/reifier/CatchReifier.java     |  81 +++
 .../org/apache/camel/reifier/ChoiceReifier.java    |  64 +++
 .../apache/camel/reifier/ClaimCheckReifier.java    | 123 +++++
 .../apache/camel/reifier/ConvertBodyReifier.java   |  58 +++
 .../org/apache/camel/reifier/DelayReifier.java     |  66 +++
 .../apache/camel/reifier/DynamicRouterReifier.java |  59 +++
 .../org/apache/camel/reifier/EnrichReifier.java    |  80 +++
 .../apache/camel/reifier/ExpressionReifier.java    |  53 ++
 .../FilterReifier.java}                            |  37 +-
 .../FinallyReifier.java}                           |  36 +-
 .../org/apache/camel/reifier/HystrixReifier.java   |  22 +-
 .../camel/reifier/IdempotentConsumerReifier.java   |  67 +++
 .../InterceptFromReifier.java}                     |  65 +--
 .../org/apache/camel/reifier/InterceptReifier.java |  72 +++
 .../reifier/InterceptSendToEndpointReifier.java    | 100 ++++
 .../apache/camel/reifier/LoadBalanceReifier.java   |  74 +++
 .../LogDefinition.java => reifier/LogReifier.java} | 155 +-----
 .../java/org/apache/camel/reifier/LoopReifier.java |  49 ++
 .../MarshalReifier.java}                           |  38 +-
 .../org/apache/camel/reifier/MulticastReifier.java | 116 +++++
 .../apache/camel/reifier/OnCompletionReifier.java  |  90 ++++
 .../apache/camel/reifier/OnExceptionReifier.java   | 164 ++++++
 .../apache/camel/reifier/OnFallbackReifier.java    |  22 +-
 .../org/apache/camel/reifier/OtherwiseReifier.java |  21 +-
 .../org/apache/camel/reifier/PipelineReifier.java  |  21 +-
 .../org/apache/camel/reifier/PolicyReifier.java    |  63 +++
 .../apache/camel/reifier/PollEnrichReifier.java    |  87 ++++
 .../org/apache/camel/reifier/ProcessReifier.java   |  56 ++
 .../org/apache/camel/reifier/ProcessorReifier.java | 579 +++++++++++++++++++++
 .../apache/camel/reifier/RecipientListReifier.java | 153 ++++++
 .../RemoveHeaderReifier.java}                      |  36 +-
 .../RemoveHeadersReifier.java}                     |  53 +-
 .../RemovePropertiesReifier.java}                  |  53 +-
 .../RemovePropertyReifier.java}                    |  36 +-
 .../apache/camel/reifier/ResequenceReifier.java    | 136 +++++
 .../org/apache/camel/reifier/RethrowReifier.java   |  25 +
 .../org/apache/camel/reifier/RollbackReifier.java  |  47 ++
 .../org/apache/camel/reifier/RouteReifier.java     | 339 ++++++++++++
 .../apache/camel/reifier/RoutingSlipReifier.java   |  59 +++
 .../java/org/apache/camel/reifier/SagaReifier.java | 113 ++++
 .../org/apache/camel/reifier/SamplingReifier.java  |  46 ++
 .../ScriptReifier.java}                            |  36 +-
 .../SendReifier.java}                              |  41 +-
 .../apache/camel/reifier/ServiceCallReifier.java   | 462 ++++++++++++++++
 .../SetBodyReifier.java}                           |  36 +-
 .../camel/reifier/SetExchangePatternReifier.java   |  22 +-
 .../SetFaultBodyReifier.java}                      |  36 +-
 .../SetHeaderReifier.java}                         |  40 +-
 .../SetPropertyReifier.java}                       |  40 +-
 .../java/org/apache/camel/reifier/SortReifier.java |  65 +++
 .../org/apache/camel/reifier/SplitReifier.java     |  99 ++++
 .../java/org/apache/camel/reifier/StopReifier.java |  22 +-
 .../org/apache/camel/reifier/ThreadsReifier.java   | 127 +++++
 .../org/apache/camel/reifier/ThrottleReifier.java  |  80 +++
 .../camel/reifier/ThrowExceptionReifier.java       |  52 ++
 .../ToDynamicReifier.java}                         | 157 +-----
 .../TransactedReifier.java}                        | 149 +-----
 .../TransformReifier.java}                         |  36 +-
 .../java/org/apache/camel/reifier/TryReifier.java  |  65 +++
 .../UnmarshalReifier.java}                         |  39 +-
 .../org/apache/camel/reifier/ValidateReifier.java  |  23 +-
 .../java/org/apache/camel/reifier/WhenReifier.java |  21 +-
 .../WhenSkipSendToEndpointReifier.java}            |  36 +-
 .../org/apache/camel/reifier/WireTapReifier.java   | 100 ++++
 .../loadbalancer/CustomLoadBalancerReifier.java}   |  35 +-
 .../loadbalancer/FailoverLoadBalancerReifier.java  |  73 +++
 .../reifier/loadbalancer/LoadBalancerReifier.java  |  86 +++
 .../loadbalancer/RandomLoadBalancerReifier.java    |  23 +-
 .../RoundRobinLoadBalancerReifier.java             |  23 +-
 .../loadbalancer/StickyLoadBalancerReifier.java    |  23 +-
 .../loadbalancer/TopicLoadBalancerReifier.java     |  23 +-
 .../loadbalancer/WeightedLoadBalancerReifier.java  |  62 +++
 .../rest/RestBindingReifier.java}                  | 276 +---------
 .../transformer/CustomTransformeReifier.java}      |  76 +--
 .../transformer/DataFormatTransformeReifier.java   |  41 ++
 .../transformer/EndpointTransformeReifier.java     |  46 ++
 .../reifier/transformer/TransformerReifier.java    |  61 +++
 .../validator/CustomValidatorReifier.java}         |  72 +--
 .../validator/EndpointValidatorReifier.java        |  44 ++
 .../validator/PredicateValidatorReifier.java       |  42 ++
 .../camel/reifier/validator/ValidatorReifier.java  |  61 +++
 .../RandomLoadBalanceJavaDSLBuilderTest.java       |   2 +-
 .../JtaTransactionErrorHandlerBuilder.java         |   4 +-
 172 files changed, 5431 insertions(+), 5668 deletions(-)

diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
index 5ab0dc1..25fdfdf 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
@@ -110,6 +110,7 @@ import org.apache.camel.model.transformer.TransformerDefinition;
 import org.apache.camel.model.validator.ValidatorDefinition;
 import org.apache.camel.processor.interceptor.Debug;
 import org.apache.camel.processor.interceptor.HandleFault;
+import org.apache.camel.reifier.RouteReifier;
 import org.apache.camel.runtimecatalog.RuntimeCamelCatalog;
 import org.apache.camel.runtimecatalog.impl.DefaultRuntimeCamelCatalog;
 import org.apache.camel.spi.AsyncProcessorAwaitManager;
@@ -1136,7 +1137,7 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
             route.prepare(this);
 
             List<Route> routes = new ArrayList<>();
-            List<RouteContext> routeContexts = route.addRoutes(this, routes);
+            List<RouteContext> routeContexts = new RouteReifier(route).addRoutes(this, routes);
             RouteService routeService = new RouteService(this, route, routeContexts, routes);
             startRouteService(routeService, true);
         } finally {
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java
index 95e334f..1f3175b 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultRouteContext.java
@@ -39,6 +39,8 @@ import org.apache.camel.processor.CamelInternalProcessor;
 import org.apache.camel.processor.CamelInternalProcessorAdvice;
 import org.apache.camel.processor.ContractAdvice;
 import org.apache.camel.processor.Pipeline;
+import org.apache.camel.reifier.RouteReifier;
+import org.apache.camel.reifier.rest.RestBindingReifier;
 import org.apache.camel.spi.Contract;
 import org.apache.camel.spi.InterceptStrategy;
 import org.apache.camel.spi.ManagementInterceptStrategy;
@@ -112,7 +114,7 @@ public class DefaultRouteContext implements RouteContext {
     }
 
     public Endpoint resolveEndpoint(String uri) {
-        return route.resolveEndpoint(getCamelContext(), uri);
+        return new RouteReifier(route).resolveEndpoint(getCamelContext(), uri);
     }
 
     public Endpoint resolveEndpoint(String uri, String ref) {
@@ -203,7 +205,7 @@ public class DefaultRouteContext implements RouteContext {
             // wrap in REST binding
             if (route.getRestBindingDefinition() != null) {
                 try {
-                    internal.addAdvice(route.getRestBindingDefinition().createRestBindingAdvice(this));
+                    internal.addAdvice(new RestBindingReifier(route.getRestBindingDefinition()).createRestBindingAdvice(this));
                 } catch (Exception e) {
                     throw RuntimeCamelException.wrapRuntimeCamelException(e);
                 }
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultTransformerRegistry.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultTransformerRegistry.java
index f039585..06a9d62 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultTransformerRegistry.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultTransformerRegistry.java
@@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.camel.CamelContext;
 import org.apache.camel.impl.transformer.TransformerKey;
 import org.apache.camel.model.transformer.TransformerDefinition;
+import org.apache.camel.reifier.transformer.TransformerReifier;
 import org.apache.camel.spi.DataType;
 import org.apache.camel.spi.Transformer;
 import org.apache.camel.spi.TransformerRegistry;
@@ -45,7 +46,7 @@ public class DefaultTransformerRegistry extends AbstractDynamicRegistry<Transfor
         super(context, CamelContextHelper.getMaximumTransformerCacheSize(context));
         this.aliasMap = new ConcurrentHashMap<>();
         for (TransformerDefinition def : definitions) {
-            Transformer transformer = def.createTransformer(context);
+            Transformer transformer = TransformerReifier.reifier(def).createTransformer(context);
             context.addService(transformer);
             put(createKey(def), transformer);
         }
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultValidatorRegistry.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultValidatorRegistry.java
index 6619000..1332801 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultValidatorRegistry.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultValidatorRegistry.java
@@ -21,6 +21,7 @@ import java.util.List;
 import org.apache.camel.CamelContext;
 import org.apache.camel.impl.validator.ValidatorKey;
 import org.apache.camel.model.validator.ValidatorDefinition;
+import org.apache.camel.reifier.validator.ValidatorReifier;
 import org.apache.camel.spi.DataType;
 import org.apache.camel.spi.Validator;
 import org.apache.camel.spi.ValidatorRegistry;
@@ -39,7 +40,7 @@ public class DefaultValidatorRegistry extends AbstractDynamicRegistry<ValidatorK
     public DefaultValidatorRegistry(CamelContext context, List<ValidatorDefinition> definitions) throws Exception {
         this(context);
         for (ValidatorDefinition def : definitions) {
-            Validator validator = def.createValidator(context);
+            Validator validator = ValidatorReifier.reifier(def).createValidator(context);
             context.addService(validator);
             put(createKey(def), validator);
         }
diff --git a/camel-core/src/main/java/org/apache/camel/model/AggregateDefinition.java b/camel-core/src/main/java/org/apache/camel/model/AggregateDefinition.java
index 3bad006..dd57145 100644
--- a/camel-core/src/main/java/org/apache/camel/model/AggregateDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/AggregateDefinition.java
@@ -29,26 +29,19 @@ import javax.xml.bind.annotation.XmlElementRef;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
 import org.apache.camel.Expression;
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.AggregationStrategyClause;
 import org.apache.camel.builder.ExpressionClause;
 import org.apache.camel.builder.PredicateClause;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.CamelInternalProcessor;
 import org.apache.camel.processor.aggregate.AggregateController;
-import org.apache.camel.processor.aggregate.AggregateProcessor;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
 import org.apache.camel.processor.aggregate.ClosedCorrelationKeyException;
 import org.apache.camel.processor.aggregate.OptimisticLockRetryPolicy;
 import org.apache.camel.spi.AggregationRepository;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.concurrent.SynchronousExecutorService;
 
 /**
  * Aggregates many messages into a single message
@@ -170,133 +163,6 @@ public class AggregateDefinition extends ProcessorDefinition<AggregateDefinition
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return createAggregator(routeContext);
-    }
-
-    protected AggregateProcessor createAggregator(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        // wrap the aggregate route in a unit of work processor
-        CamelInternalProcessor internal = new CamelInternalProcessor(childProcessor);
-        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
-
-        Expression correlation = getExpression().createExpression(routeContext);
-        AggregationStrategy strategy = createAggregationStrategy(routeContext);
-
-        boolean parallel = getParallelProcessing() != null && getParallelProcessing();
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, parallel);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "Aggregator", this, parallel);
-        if (threadPool == null && !parallel) {
-            // executor service is mandatory for the Aggregator
-            // we do not run in parallel mode, but use a synchronous executor, so we run in current thread
-            threadPool = new SynchronousExecutorService();
-            shutdownThreadPool = true;
-        }
-
-        AggregateProcessor answer = new AggregateProcessor(routeContext.getCamelContext(), internal,
-                correlation, strategy, threadPool, shutdownThreadPool);
-
-        AggregationRepository repository = createAggregationRepository(routeContext);
-        if (repository != null) {
-            answer.setAggregationRepository(repository);
-        }
-
-        if (getAggregateController() == null && getAggregateControllerRef() != null) {
-            setAggregateController(routeContext.mandatoryLookup(getAggregateControllerRef(), AggregateController.class));
-        }
-
-        // this EIP supports using a shared timeout checker thread pool or fallback to create a new thread pool
-        boolean shutdownTimeoutThreadPool = false;
-        ScheduledExecutorService timeoutThreadPool = timeoutCheckerExecutorService;
-        if (timeoutThreadPool == null && timeoutCheckerExecutorServiceRef != null) {
-            // lookup existing thread pool
-            timeoutThreadPool = routeContext.getCamelContext().getRegistry().lookupByNameAndType(timeoutCheckerExecutorServiceRef, ScheduledExecutorService.class);
-            if (timeoutThreadPool == null) {
-                // then create a thread pool assuming the ref is a thread pool profile id
-                timeoutThreadPool = routeContext.getCamelContext().getExecutorServiceManager().newScheduledThreadPool(this,
-                        AggregateProcessor.AGGREGATE_TIMEOUT_CHECKER, timeoutCheckerExecutorServiceRef);
-                if (timeoutThreadPool == null) {
-                    throw new IllegalArgumentException("ExecutorServiceRef " + timeoutCheckerExecutorServiceRef 
-                            + " not found in registry (as an ScheduledExecutorService instance) or as a thread pool profile.");
-                }
-                shutdownTimeoutThreadPool = true;
-            }
-        }
-        answer.setTimeoutCheckerExecutorService(timeoutThreadPool);
-        answer.setShutdownTimeoutCheckerExecutorService(shutdownTimeoutThreadPool);
-
-        // set other options
-        answer.setParallelProcessing(parallel);
-        if (getOptimisticLocking() != null) {
-            answer.setOptimisticLocking(getOptimisticLocking());
-        }
-        if (getCompletionPredicate() != null) {
-            Predicate predicate = getCompletionPredicate().createPredicate(routeContext);
-            answer.setCompletionPredicate(predicate);
-        } else if (strategy instanceof Predicate) {
-            // if aggregation strategy implements predicate and was not configured then use as fallback
-            log.debug("Using AggregationStrategy as completion predicate: {}", strategy);
-            answer.setCompletionPredicate((Predicate) strategy);
-        }
-        if (getCompletionTimeoutExpression() != null) {
-            Expression expression = getCompletionTimeoutExpression().createExpression(routeContext);
-            answer.setCompletionTimeoutExpression(expression);
-        }
-        if (getCompletionTimeout() != null) {
-            answer.setCompletionTimeout(getCompletionTimeout());
-        }
-        if (getCompletionInterval() != null) {
-            answer.setCompletionInterval(getCompletionInterval());
-        }
-        if (getCompletionSizeExpression() != null) {
-            Expression expression = getCompletionSizeExpression().createExpression(routeContext);
-            answer.setCompletionSizeExpression(expression);
-        }
-        if (getCompletionSize() != null) {
-            answer.setCompletionSize(getCompletionSize());
-        }
-        if (getCompletionFromBatchConsumer() != null) {
-            answer.setCompletionFromBatchConsumer(getCompletionFromBatchConsumer());
-        }
-        if (getCompletionOnNewCorrelationGroup() != null) {
-            answer.setCompletionOnNewCorrelationGroup(getCompletionOnNewCorrelationGroup());
-        }
-        if (getEagerCheckCompletion() != null) {
-            answer.setEagerCheckCompletion(getEagerCheckCompletion());
-        }
-        if (getIgnoreInvalidCorrelationKeys() != null) {
-            answer.setIgnoreInvalidCorrelationKeys(getIgnoreInvalidCorrelationKeys());
-        }
-        if (getCloseCorrelationKeyOnCompletion() != null) {
-            answer.setCloseCorrelationKeyOnCompletion(getCloseCorrelationKeyOnCompletion());
-        }
-        if (getDiscardOnCompletionTimeout() != null) {
-            answer.setDiscardOnCompletionTimeout(getDiscardOnCompletionTimeout());
-        }
-        if (getForceCompletionOnStop() != null) {
-            answer.setForceCompletionOnStop(getForceCompletionOnStop());
-        }
-        if (getCompleteAllOnStop() != null) {
-            answer.setCompleteAllOnStop(getCompleteAllOnStop());
-        }
-        if (optimisticLockRetryPolicy == null) {
-            if (getOptimisticLockRetryPolicyDefinition() != null) {
-                answer.setOptimisticLockRetryPolicy(getOptimisticLockRetryPolicyDefinition().createOptimisticLockRetryPolicy());
-            }
-        } else {
-            answer.setOptimisticLockRetryPolicy(optimisticLockRetryPolicy);
-        }
-        if (getAggregateController() != null) {
-            answer.setAggregateController(getAggregateController());
-        }
-        if (getCompletionTimeoutCheckerInterval() != null) {
-            answer.setCompletionTimeoutCheckerInterval(getCompletionTimeoutCheckerInterval());
-        }
-        return answer;
-    }
-
-    @Override
     public void configureChild(ProcessorDefinition<?> output) {
         if (expression instanceof ExpressionClause) {
             ExpressionClause<?> clause = (ExpressionClause<?>) expression;
@@ -314,43 +180,6 @@ public class AggregateDefinition extends ProcessorDefinition<AggregateDefinition
         }
     }
 
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && strategyRef != null) {
-            Object aggStrategy = routeContext.lookup(strategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getAggregationStrategyMethodName());
-                if (getStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + strategyRef);
-            }
-        }
-
-        if (strategy == null) {
-            throw new IllegalArgumentException("AggregationStrategy or AggregationStrategyRef must be set on " + this);
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        return strategy;
-    }
-
-    private AggregationRepository createAggregationRepository(RouteContext routeContext) {
-        AggregationRepository repository = getAggregationRepository();
-        if (repository == null && aggregationRepositoryRef != null) {
-            repository = routeContext.mandatoryLookup(aggregationRepositoryRef, AggregationRepository.class);
-        }
-        return repository;
-    }
-
     public AggregationStrategy getAggregationStrategy() {
         return aggregationStrategy;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java b/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java
index 9557204..b03c05b 100644
--- a/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java
@@ -22,20 +22,7 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.component.bean.BeanHolder;
-import org.apache.camel.component.bean.BeanInfo;
-import org.apache.camel.component.bean.BeanProcessor;
-import org.apache.camel.component.bean.ConstantBeanHolder;
-import org.apache.camel.component.bean.ConstantStaticTypeBeanHolder;
-import org.apache.camel.component.bean.ConstantTypeBeanHolder;
-import org.apache.camel.component.bean.MethodNotFoundException;
-import org.apache.camel.component.bean.RegistryBean;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Calls a java bean
@@ -131,6 +118,10 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
         this.bean = bean;
     }
 
+    public Object getBean() {
+        return bean;
+    }
+
     public String getBeanType() {
         return beanType;
     }
@@ -142,6 +133,10 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
         this.beanType = beanType;
     }
 
+    public Class<?> getBeanClass() {
+        return beanClass;
+    }
+
     /**
      * Sets the Class of the bean
      */
@@ -163,114 +158,4 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
     // Fluent API
     //-------------------------------------------------------------------------
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        BeanProcessor answer;
-        Class<?> clazz = bean != null ? bean.getClass() : null;
-        BeanHolder beanHolder;
-
-        if (ObjectHelper.isNotEmpty(ref)) {
-            // lets cache by default
-            if (isCacheBean()) {
-                // cache the registry lookup which avoids repeat lookup in the registry
-                beanHolder = new RegistryBean(routeContext.getCamelContext(), ref).createCacheHolder();
-                // bean holder will check if the bean exists
-                bean = beanHolder.getBean();
-            } else {
-                // we do not cache so we invoke on-demand
-                beanHolder = new RegistryBean(routeContext.getCamelContext(), ref);
-            }
-            answer = new BeanProcessor(beanHolder);
-        } else {
-            if (bean == null) {
-                
-                if (beanType == null && beanClass == null) {
-                    throw new IllegalArgumentException("bean, ref or beanType must be provided");
-                }
-
-                // the clazz is either from beanType or beanClass
-                if (beanType != null) {
-                    try {
-                        clazz = routeContext.getCamelContext().getClassResolver().resolveMandatoryClass(beanType);
-                    } catch (ClassNotFoundException e) {
-                        throw RuntimeCamelException.wrapRuntimeCamelException(e);
-                    }
-                } else {
-                    clazz = beanClass;
-                }
-
-                // attempt to create bean using injector which supports auto-wiring
-                if (isCacheBean() && routeContext.getCamelContext().getInjector().supportsAutoWiring()) {
-                    try {
-                        log.debug("Attempting to create new bean instance from class: {} via auto-wiring enabled", clazz);
-                        bean = CamelContextHelper.newInstance(routeContext.getCamelContext(), clazz);
-                    } catch (Throwable e) {
-                        log.debug("Error creating new bean instance from class: " + clazz + ". This exception is ignored", e);
-                    }
-                }
-
-                // create a bean if there is a default public no-arg constructor
-                if (bean == null && isCacheBean() && ObjectHelper.hasDefaultPublicNoArgConstructor(clazz)) {
-                    log.debug("Class has default no-arg constructor so creating a new bean instance: {}", clazz);
-                    bean = CamelContextHelper.newInstance(routeContext.getCamelContext(), clazz);
-                    ObjectHelper.notNull(bean, "bean", this);
-                }
-            }
-
-            // validate the bean type is not from java so you by mistake think its a reference
-            // to a bean name but the String is being invoke instead
-            if (bean instanceof String) {
-                throw new IllegalArgumentException("The bean instance is a java.lang.String type: " + bean
-                    + ". We suppose you want to refer to a bean instance by its id instead. Please use ref.");
-            }
-
-            // the holder should either be bean or type based
-            if (bean != null) {
-                beanHolder = new ConstantBeanHolder(bean, routeContext.getCamelContext());
-            } else {
-                if (isCacheBean() && ObjectHelper.hasDefaultPublicNoArgConstructor(clazz)) {
-                    // we can only cache if we can create an instance of the bean, and for that we need a public constructor
-                    beanHolder = new ConstantTypeBeanHolder(clazz, routeContext.getCamelContext()).createCacheHolder();
-                } else {
-                    if (ObjectHelper.hasDefaultPublicNoArgConstructor(clazz)) {
-                        beanHolder = new ConstantTypeBeanHolder(clazz, routeContext.getCamelContext());
-                    } else {
-                        // this is only for invoking static methods on the bean
-                        beanHolder = new ConstantStaticTypeBeanHolder(clazz, routeContext.getCamelContext());
-                    }
-                }
-            }
-            answer = new BeanProcessor(beanHolder);
-        }
-        
-        // check for method exists
-        if (method != null) {
-            answer.setMethod(method);
-
-            // check there is a method with the given name, and leverage BeanInfo for that
-            // which we only do if we are caching the bean as otherwise we will create a bean instance for this check
-            // which we only want to do if we cache the bean
-            if (isCacheBean()) {
-                BeanInfo beanInfo = beanHolder.getBeanInfo();
-                if (bean != null) {
-                    // there is a bean instance, so check for any methods
-                    if (!beanInfo.hasMethod(method)) {
-                        throw RuntimeCamelException.wrapRuntimeCamelException(new MethodNotFoundException(null, bean, method));
-                    }
-                } else if (clazz != null) {
-                    // there is no bean instance, so check for static methods only
-                    if (!beanInfo.hasStaticMethod(method)) {
-                        throw RuntimeCamelException.wrapRuntimeCamelException(new MethodNotFoundException(null, clazz, method, true));
-                    }
-                }
-            }
-        }
-
-        return answer;
-    }
-
-    private boolean isCacheBean() {
-        return cache == null || cache;
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/CatchDefinition.java b/camel-core/src/main/java/org/apache/camel/model/CatchDefinition.java
index 41e9780..dccc118 100644
--- a/camel-core/src/main/java/org/apache/camel/model/CatchDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/CatchDefinition.java
@@ -26,13 +26,9 @@ import javax.xml.bind.annotation.XmlElementRef;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.CatchProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Catches exceptions as part of a try, catch, finally block
@@ -82,39 +78,6 @@ public class CatchDefinition extends ProcessorDefinition<CatchDefinition> {
     }
 
     @Override
-    public CatchProcessor createProcessor(RouteContext routeContext) throws Exception {
-        // create and load exceptions if not done
-        if (exceptionClasses == null) {
-            exceptionClasses = createExceptionClasses(routeContext.getCamelContext());
-        }
-
-        // must have at least one exception
-        if (exceptionClasses.isEmpty()) {
-            throw new IllegalArgumentException("At least one Exception must be configured to catch");
-        }
-
-        // parent must be a try
-        if (!(getParent() instanceof TryDefinition)) {
-            throw new IllegalArgumentException("This doCatch should have a doTry as its parent on " + this);
-        }
-
-        // do catch does not mandate a child processor
-        Processor childProcessor = this.createChildProcessor(routeContext, false);
-
-        Predicate when = null;
-        if (onWhen != null) {
-            when = onWhen.getExpression().createPredicate(routeContext);
-        }
-
-        Predicate handle = handledPolicy;
-        if (handled != null) {
-            handle = handled.createPredicate(routeContext);
-        }
-
-        return new CatchProcessor(exceptionClasses, childProcessor, when, handle);
-    }
-
-    @Override
     public List<ProcessorDefinition<?>> getOutputs() {
         return outputs;
     }
@@ -224,15 +187,4 @@ public class CatchDefinition extends ProcessorDefinition<CatchDefinition> {
         this.handled = handled;
     }
 
-    protected List<Class<? extends Throwable>> createExceptionClasses(CamelContext context) throws ClassNotFoundException {
-        // must use the class resolver from CamelContext to load classes to ensure it can
-        // be loaded in all kind of environments such as JEE servers and OSGi etc.
-        List<String> list = getExceptions();
-        List<Class<? extends Throwable>> answer = new ArrayList<>(list.size());
-        for (String name : list) {
-            Class<Throwable> type = context.getClassResolver().resolveMandatoryClass(name, Throwable.class);
-            answer.add(type);
-        }
-        return answer;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ChoiceDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ChoiceDefinition.java
index 7df7d3c..eea5789 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ChoiceDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ChoiceDefinition.java
@@ -27,14 +27,9 @@ import javax.xml.bind.annotation.XmlElementRef;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionClause;
-import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.ChoiceProcessor;
-import org.apache.camel.processor.FilterProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.CollectionStringBuffer;
 import org.apache.camel.util.ObjectHelper;
 
@@ -130,31 +125,6 @@ public class ChoiceDefinition extends ProcessorDefinition<ChoiceDefinition> {
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        List<FilterProcessor> filters = new ArrayList<>();
-        for (WhenDefinition whenClause : whenClauses) {
-            // also resolve properties and constant fields on embedded expressions in the when clauses
-            ExpressionNode exp = whenClause;
-            ExpressionDefinition expressionDefinition = exp.getExpression();
-            if (expressionDefinition != null) {
-                // resolve properties before we create the processor
-                ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
-
-                // resolve constant fields (eg Exchange.FILE_NAME)
-                ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
-            }
-
-            FilterProcessor filter = (FilterProcessor) createProcessor(routeContext, whenClause);
-            filters.add(filter);
-        }
-        Processor otherwiseProcessor = null;
-        if (otherwise != null) {
-            otherwiseProcessor = createProcessor(routeContext, otherwise);
-        }
-        return new ChoiceProcessor(filters, otherwiseProcessor);
-    }
-
-    @Override
     public void addOutput(ProcessorDefinition<?> output) {
         if (onlyWhenOrOtherwise) {
             if (output instanceof WhenDefinition || output instanceof OtherwiseDefinition) {
diff --git a/camel-core/src/main/java/org/apache/camel/model/ClaimCheckDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ClaimCheckDefinition.java
index 4896f5a..334ad6c 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ClaimCheckDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ClaimCheckDefinition.java
@@ -22,15 +22,9 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.ClaimCheckProcessor;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.EndpointHelper;
-import org.apache.camel.support.ObjectHelper;
 
 /**
  * The Claim Check EIP allows you to replace message content with a claim check (a unique key),
@@ -76,92 +70,6 @@ public class ClaimCheckDefinition extends NoOutputDefinition<ClaimCheckDefinitio
         return "claimCheck";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        org.apache.camel.util.ObjectHelper.notNull(operation, "operation", this);
-
-        ClaimCheckProcessor claim = new ClaimCheckProcessor();
-        claim.setOperation(operation.name());
-        claim.setKey(getKey());
-        claim.setFilter(getFilter());
-
-        AggregationStrategy strategy = createAggregationStrategy(routeContext);
-        if (strategy != null) {
-            claim.setAggregationStrategy(strategy);
-        }
-
-        // only filter or aggregation strategy can be configured not both
-        if (getFilter() != null && strategy != null) {
-            throw new IllegalArgumentException("Cannot use both filter and custom aggregation strategy on ClaimCheck EIP");
-        }
-
-        // validate filter, we cannot have both +/- at the same time
-        if (getFilter() != null) {
-            Iterable<?> it = ObjectHelper.createIterable(filter, ",");
-            boolean includeBody = false;
-            boolean excludeBody = false;
-            for (Object o : it) {
-                String pattern = o.toString();
-                if ("body".equals(pattern) || "+body".equals(pattern)) {
-                    includeBody = true;
-                } else if ("-body".equals(pattern)) {
-                    excludeBody = true;
-                }
-            }
-            if (includeBody && excludeBody) {
-                throw new IllegalArgumentException("Cannot have both include and exclude body at the same time in the filter: " + filter);
-            }
-            boolean includeHeaders = false;
-            boolean excludeHeaders = false;
-            for (Object o : it) {
-                String pattern = o.toString();
-                if ("headers".equals(pattern) || "+headers".equals(pattern)) {
-                    includeHeaders = true;
-                } else if ("-headers".equals(pattern)) {
-                    excludeHeaders = true;
-                }
-            }
-            if (includeHeaders && excludeHeaders) {
-                throw new IllegalArgumentException("Cannot have both include and exclude headers at the same time in the filter: " + filter);
-            }
-            boolean includeHeader = false;
-            boolean excludeHeader = false;
-            for (Object o : it) {
-                String pattern = o.toString();
-                if (pattern.startsWith("header:") || pattern.startsWith("+header:")) {
-                    includeHeader = true;
-                } else if (pattern.startsWith("-header:")) {
-                    excludeHeader = true;
-                }
-            }
-            if (includeHeader && excludeHeader) {
-                throw new IllegalArgumentException("Cannot have both include and exclude header at the same time in the filter: " + filter);
-            }
-        }
-
-        return claim;
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && aggregationStrategyRef != null) {
-            Object aggStrategy = routeContext.lookup(aggregationStrategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                strategy = new AggregationStrategyBeanAdapter(aggStrategy, getAggregationStrategyMethodName());
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + aggregationStrategyRef);
-            }
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        return strategy;
-    }
-
     // Fluent API
     //-------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/ConvertBodyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ConvertBodyDefinition.java
index 09b8b30..dcedc9b 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ConvertBodyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ConvertBodyDefinition.java
@@ -16,19 +16,13 @@
  */
 package org.apache.camel.model;
 
-import java.nio.charset.Charset;
-import java.nio.charset.UnsupportedCharsetException;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.ConvertBodyProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Converts the message body to another type
@@ -77,30 +71,6 @@ public class ConvertBodyDefinition extends NoOutputDefinition<ConvertBodyDefinit
         return "convertBodyTo[" + getType() + "]";
     }
     
-    public static void validateCharset(String charset) throws UnsupportedCharsetException {
-        if (charset != null) {
-            if (Charset.isSupported(charset)) {
-                Charset.forName(charset);
-                return;
-            }
-        }
-        throw new UnsupportedCharsetException(charset);
-    }
-
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        if (typeClass == null && type != null) {
-            typeClass = routeContext.getCamelContext().getClassResolver().resolveMandatoryClass(type);
-        }
-
-        // validate charset
-        if (charset != null) {
-            validateCharset(charset);
-        }
-
-        return new ConvertBodyProcessor(getTypeClass(), getCharset());
-    }
-
     public String getType() {
         return type;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/DelayDefinition.java b/camel-core/src/main/java/org/apache/camel/model/DelayDefinition.java
index 64ec97e..574de03 100644
--- a/camel-core/src/main/java/org/apache/camel/model/DelayDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/DelayDefinition.java
@@ -17,7 +17,6 @@
 package org.apache.camel.model;
 
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ScheduledExecutorService;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -26,12 +25,9 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionBuilder;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.Delayer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Delays processing for a specified length of time
@@ -74,36 +70,6 @@ public class DelayDefinition extends ExpressionNode implements ExecutorServiceAw
         return "Delay[" + getExpression() + " -> " + getOutputs() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, false);
-        Expression delay = createAbsoluteTimeDelayExpression(routeContext);
-
-        boolean async = getAsyncDelayed() != null && getAsyncDelayed();
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, async);
-        ScheduledExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredScheduledExecutorService(routeContext, "Delay", this, async);
-
-        Delayer answer = new Delayer(routeContext.getCamelContext(), childProcessor, delay, threadPool, shutdownThreadPool);
-        if (getAsyncDelayed() != null) {
-            answer.setAsyncDelayed(getAsyncDelayed());
-        }
-        if (getCallerRunsWhenRejected() == null) {
-            // should be default true
-            answer.setCallerRunsWhenRejected(true);
-        } else {
-            answer.setCallerRunsWhenRejected(getCallerRunsWhenRejected());
-        }
-        return answer;
-    }
-
-    private Expression createAbsoluteTimeDelayExpression(RouteContext routeContext) {
-        ExpressionDefinition expr = getExpression();
-        if (expr != null) {
-            return expr.createExpression(routeContext);
-        }
-        return null;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/DynamicRouterDefinition.java b/camel-core/src/main/java/org/apache/camel/model/DynamicRouterDefinition.java
index f2307f6..a4c2073 100644
--- a/camel-core/src/main/java/org/apache/camel/model/DynamicRouterDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/DynamicRouterDefinition.java
@@ -24,15 +24,9 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.AsyncProcessor;
-import org.apache.camel.ErrorHandlerFactory;
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
-import org.apache.camel.impl.DefaultProducerCache;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.DynamicRouter;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Routes messages based on dynamic rules
@@ -78,30 +72,6 @@ public class DynamicRouterDefinition<Type extends ProcessorDefinition<Type>> ext
         return Collections.emptyList();
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expression = getExpression().createExpression(routeContext);
-        String delimiter = getUriDelimiter() != null ? getUriDelimiter() : DEFAULT_DELIMITER;
-
-        DynamicRouter dynamicRouter = new DynamicRouter(routeContext.getCamelContext(), expression, delimiter);
-        if (getIgnoreInvalidEndpoints() != null) {
-            dynamicRouter.setIgnoreInvalidEndpoints(getIgnoreInvalidEndpoints());
-        }
-        if (getCacheSize() != null) {
-            dynamicRouter.setCacheSize(getCacheSize());
-        }
-
-        // and wrap this in an error handler
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        ErrorHandlerFactory builder = route.getErrorHandlerBuilder();
-        // create error handler (create error handler directly to keep it light weight,
-        // instead of using ProcessorDefinition.wrapInErrorHandler)
-        AsyncProcessor errorHandler = (AsyncProcessor) builder.createErrorHandler(routeContext, dynamicRouter.newRoutingSlipProcessorForErrorHandler());
-        dynamicRouter.setErrorHandler(errorHandler);
-
-        return dynamicRouter;
-    }
-
     /**
      * Expression to call that returns the endpoint(s) to route to in the dynamic routing.
      * <p/>
diff --git a/camel-core/src/main/java/org/apache/camel/model/EnrichDefinition.java b/camel-core/src/main/java/org/apache/camel/model/EnrichDefinition.java
index 95011d2..fc8c1d0 100644
--- a/camel-core/src/main/java/org/apache/camel/model/EnrichDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/EnrichDefinition.java
@@ -22,21 +22,14 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
-import org.apache.camel.Expression;
-import org.apache.camel.Processor;
-import org.apache.camel.impl.DefaultProducerCache;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.Enricher;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Enriches a message with data from a secondary resource
  *
- * @see Enricher
+ * @see org.apache.camel.processor.Enricher
  */
 @Metadata(label = "eip,transformation")
 @XmlRootElement(name = "enrich")
@@ -82,51 +75,6 @@ public class EnrichDefinition extends NoOutputExpressionNode {
         return "enrich[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-
-        Expression exp = getExpression().createExpression(routeContext);
-        boolean isShareUnitOfWork = getShareUnitOfWork() != null && getShareUnitOfWork();
-        boolean isIgnoreInvalidEndpoint = getIgnoreInvalidEndpoint() != null && getIgnoreInvalidEndpoint();
-
-        Enricher enricher = new Enricher(exp);
-        enricher.setShareUnitOfWork(isShareUnitOfWork);
-        enricher.setIgnoreInvalidEndpoint(isIgnoreInvalidEndpoint);
-        AggregationStrategy strategy = createAggregationStrategy(routeContext);
-        if (strategy != null) {
-            enricher.setAggregationStrategy(strategy);
-        }
-        if (aggregateOnException != null) {
-            enricher.setAggregateOnException(aggregateOnException);
-        }
-        return enricher;
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && aggregationStrategyRef != null) {
-            Object aggStrategy = routeContext.lookup(aggregationStrategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getAggregationStrategyMethodName());
-                if (getAggregationStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getAggregationStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getAggregationStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + aggregationStrategyRef);
-            }
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        return strategy;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/ExpressionNode.java b/camel-core/src/main/java/org/apache/camel/model/ExpressionNode.java
index bf46b3c..edd0dd9 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ExpressionNode.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ExpressionNode.java
@@ -26,11 +26,8 @@ import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Expression;
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionClause;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.FilterProcessor;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * A base class for nodes which contain an expression and a number of outputs
@@ -93,28 +90,6 @@ public abstract class ExpressionNode extends ProcessorDefinition<ExpressionNode>
         return getExpression().getLabel();
     }
 
-    /**
-     * Creates the {@link FilterProcessor} from the expression node.
-     *
-     * @param routeContext  the route context
-     * @return the created {@link FilterProcessor}
-     * @throws Exception is thrown if error creating the processor
-     */
-    protected FilterProcessor createFilterProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = createOutputsProcessor(routeContext);
-        return new FilterProcessor(createPredicate(routeContext), childProcessor);
-    }
-
-    /**
-     * Creates the {@link Predicate} from the expression node.
-     *
-     * @param routeContext  the route context
-     * @return the created predicate
-     */
-    protected Predicate createPredicate(RouteContext routeContext) {
-        return getExpression().createPredicate(routeContext);
-    }
-
     @Override
     public void configureChild(ProcessorDefinition<?> output) {
         // reuse the logic from pre create processor
@@ -122,10 +97,10 @@ public abstract class ExpressionNode extends ProcessorDefinition<ExpressionNode>
     }
 
     @Override
-    protected void preCreateProcessor() {
-        Expression exp = expression;
-        if (expression != null && expression.getExpressionValue() != null) {
-            exp = expression.getExpressionValue();
+    public void preCreateProcessor() {
+        Expression exp = getExpression();
+        if (getExpression() != null && getExpression().getExpressionValue() != null) {
+            exp = getExpression().getExpressionValue();
         }
 
         if (exp instanceof ExpressionClause) {
@@ -135,17 +110,18 @@ public abstract class ExpressionNode extends ProcessorDefinition<ExpressionNode>
                 // ExpressionClause which is a fancy builder to define expressions and predicates
                 // using fluent builders in the DSL. However we need afterwards a callback to
                 // reset the expression to the expression type the ExpressionClause did build for us
-                expression = clause.getExpressionType();
+                setExpression(clause.getExpressionType());
             }
         }
 
-        if (expression != null && expression.getExpression() == null) {
+        if (getExpression() != null && getExpression().getExpression() == null) {
             // use toString from predicate or expression so we have some information to show in the route model
-            if (expression.getPredicate() != null) {
-                expression.setExpression(expression.getPredicate().toString());
-            } else if (expression.getExpressionValue() != null) {
-                expression.setExpression(expression.getExpressionValue().toString());
+            if (getExpression().getPredicate() != null) {
+                getExpression().setExpression(getExpression().getPredicate().toString());
+            } else if (getExpression().getExpressionValue() != null) {
+                getExpression().setExpression(getExpression().getExpressionValue().toString());
             }
         }
     }
+
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/FilterDefinition.java b/camel-core/src/main/java/org/apache/camel/model/FilterDefinition.java
index f1c1e3b..9bd2c2e 100644
--- a/camel-core/src/main/java/org/apache/camel/model/FilterDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/FilterDefinition.java
@@ -21,12 +21,9 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.FilterProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Filter out messages based using a predicate
@@ -62,18 +59,6 @@ public class FilterDefinition extends ExpressionNode {
         return "filter[" + getExpression() + "]";
     }
     
-    @Override
-    public FilterProcessor createProcessor(RouteContext routeContext) throws Exception {
-        return createFilterProcessor(routeContext);
-    }
-    
-    @Override
-    protected FilterProcessor createFilterProcessor(RouteContext routeContext) throws Exception {
-        // filter EIP should have child outputs
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-        return new FilterProcessor(createPredicate(routeContext), childProcessor);
-    }
-
     /**
      * Expression to determine if the message should be filtered or not. If the expression returns an empty value or <tt>false</tt>
      * then the message is filtered (dropped), otherwise the message is continued being routed.
diff --git a/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java
index 8e7a943..4db9ee1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java
@@ -20,10 +20,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.FinallyProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Path traversed when a try, catch, finally block exits
@@ -48,14 +45,4 @@ public class FinallyDefinition extends OutputDefinition<FinallyDefinition> {
         return "doFinally";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // parent must be a try
-        if (!(getParent() instanceof TryDefinition)) {
-            throw new IllegalArgumentException("This doFinally should have a doTry as its parent on " + this);
-        }
-
-        // do finally does mandate a child processor
-        return new FinallyProcessor(this.createChildProcessor(routeContext, false));
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/HystrixDefinition.java b/camel-core/src/main/java/org/apache/camel/model/HystrixDefinition.java
index ed024c4..6271dda 100644
--- a/camel-core/src/main/java/org/apache/camel/model/HystrixDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/HystrixDefinition.java
@@ -28,9 +28,7 @@ import javax.xml.bind.annotation.XmlElementRef;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Hystrix Circuit Breaker EIP
@@ -67,11 +65,6 @@ public class HystrixDefinition extends ProcessorDefinition<HystrixDefinition> {
         return "hystrix";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        throw new IllegalStateException("Cannot find camel-hystrix on the classpath.");
-    }
-
     public List<ProcessorDefinition<?>> getOutputs() {
         return outputs;
     }
@@ -111,7 +104,7 @@ public class HystrixDefinition extends ProcessorDefinition<HystrixDefinition> {
         return super.end();
     }
 
-    protected void preCreateProcessor() {
+    public void preCreateProcessor() {
         // move the fallback from outputs to fallback which we need to ensure
         // such as when using the XML DSL
         Iterator<ProcessorDefinition<?>> it = outputs.iterator();
diff --git a/camel-core/src/main/java/org/apache/camel/model/IdempotentConsumerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/IdempotentConsumerDefinition.java
index 5c84ae9..82d983a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/IdempotentConsumerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/IdempotentConsumerDefinition.java
@@ -23,13 +23,9 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.idempotent.IdempotentConsumer;
 import org.apache.camel.spi.IdempotentRepository;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Filters out duplicate messages
@@ -221,37 +217,4 @@ public class IdempotentConsumerDefinition extends ExpressionNode {
         this.completionEager = completionEager;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        IdempotentRepository<String> idempotentRepository = resolveMessageIdRepository(routeContext);
-        ObjectHelper.notNull(idempotentRepository, "idempotentRepository", this);
-
-        Expression expression = getExpression().createExpression(routeContext);
-
-        // these boolean should be true by default
-        boolean eager = getEager() == null || getEager();
-        boolean duplicate = getSkipDuplicate() == null || getSkipDuplicate();
-        boolean remove = getRemoveOnFailure() == null || getRemoveOnFailure();
-
-        // these boolean should be false by default
-        boolean completionEager = getCompletionEager() != null && getCompletionEager();
-
-        return new IdempotentConsumer(expression, idempotentRepository, eager, completionEager, duplicate, remove, childProcessor);
-    }
-
-    /**
-     * Strategy method to resolve the {@link org.apache.camel.spi.IdempotentRepository} to use
-     *
-     * @param routeContext route context
-     * @return the repository
-     */
-    @SuppressWarnings("unchecked")
-    protected <T> IdempotentRepository<T> resolveMessageIdRepository(RouteContext routeContext) {
-        if (messageIdRepositoryRef != null) {
-            idempotentRepository = routeContext.mandatoryLookup(messageIdRepositoryRef, IdempotentRepository.class);
-        }
-        return (IdempotentRepository<T>)idempotentRepository;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java b/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java
index a6ebd76..1a265c1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/InterceptDefinition.java
@@ -24,15 +24,10 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.NamedNode;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
-import org.apache.camel.processor.Pipeline;
 import org.apache.camel.spi.AsPredicate;
-import org.apache.camel.spi.InterceptStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Intercepts a message at each step in the route
@@ -42,8 +37,6 @@ import org.apache.camel.spi.RouteContext;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class InterceptDefinition extends OutputDefinition<InterceptDefinition> {
     @XmlTransient
-    protected Processor output;
-    @XmlTransient
     protected final List<Processor> intercepted = new ArrayList<>();
 
     public InterceptDefinition() {
@@ -74,43 +67,6 @@ public class InterceptDefinition extends OutputDefinition<InterceptDefinition> {
         return true;
     }
 
-    @Override
-    public Processor createProcessor(final RouteContext routeContext) throws Exception {
-        // create the output processor
-        output = this.createChildProcessor(routeContext, true);
-
-        // add the output as a intercept strategy to the route context so its invoked on each processing step
-        routeContext.getInterceptStrategies().add(new InterceptStrategy() {
-            private Processor interceptedTarget;
-
-            public Processor wrapProcessorInInterceptors(CamelContext context, NamedNode definition,
-                                                         Processor target, Processor nextTarget) throws Exception {
-                // store the target we are intercepting
-                this.interceptedTarget = target;
-
-                // remember the target that was intercepted
-                intercepted.add(interceptedTarget);
-
-                if (interceptedTarget != null) {
-                    // wrap in a pipeline so we continue routing to the next
-                    return Pipeline.newInstance(context, output, interceptedTarget);
-                } else {
-                    return output;
-                }
-            }
-
-            @Override
-            public String toString() {
-                return "intercept[" + (interceptedTarget != null ? interceptedTarget : output) + "]";
-            }
-        });
-
-        // remove me from the route so I am not invoked in a regular route path
-        ((RouteDefinition) routeContext.getRoute()).getOutputs().remove(this);
-        // and return no processor to invoke next from me
-        return null;
-    }
-
     /**
      * Applies this interceptor only if the given predicate is true
      *
@@ -151,6 +107,10 @@ public class InterceptDefinition extends OutputDefinition<InterceptDefinition> {
         }
     }
 
+    public List<Processor> getIntercepted() {
+        return intercepted;
+    }
+
     public Processor getInterceptedProcessor(int index) {
         // avoid out of bounds
         if (index <= intercepted.size() - 1) {
diff --git a/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java b/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
index ffcf3ef..c945ae1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
@@ -21,11 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.ExpressionAdapter;
 
 /**
  * Intercepts incoming messages
@@ -69,29 +65,6 @@ public class InterceptFromDefinition extends InterceptDefinition {
         return true;
     }
 
-    @Override
-    @SuppressWarnings({"unchecked", "rawtypes"})
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // insert a set header definition so we can set the intercepted endpoint uri as a header
-        // this allows us to use the same header for both the interceptFrom and interceptSendToEndpoint
-        SetHeaderDefinition headerDefinition = new SetHeaderDefinition(Exchange.INTERCEPTED_ENDPOINT, new ExpressionAdapter() {
-            public Object evaluate(Exchange exchange, Class type) {
-                if (exchange.getFromEndpoint() != null) {
-                    return exchange.getFromEndpoint().getEndpointUri();
-                } else {
-                    return null;
-                }
-            }
-
-            public String toString() {
-                return "";
-            }
-        });
-        getOutputs().add(0, headerDefinition);
-
-        return this.createChildProcessor(routeContext, true);
-    }
-
     public String getUri() {
         return uri;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/InterceptSendToEndpointDefinition.java b/camel-core/src/main/java/org/apache/camel/model/InterceptSendToEndpointDefinition.java
index 88cc783..44f68a8 100644
--- a/camel-core/src/main/java/org/apache/camel/model/InterceptSendToEndpointDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/InterceptSendToEndpointDefinition.java
@@ -16,25 +16,14 @@
  */
 package org.apache.camel.model;
 
-import java.util.List;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
-import org.apache.camel.impl.InterceptSendToEndpoint;
-import org.apache.camel.processor.InterceptEndpointProcessor;
 import org.apache.camel.spi.AsPredicate;
-import org.apache.camel.spi.EndpointStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.EndpointHelper;
-import org.apache.camel.util.URISupport;
 
 /**
  * Intercepts messages being sent to an endpoint
@@ -88,66 +77,6 @@ public class InterceptSendToEndpointDefinition extends OutputDefinition<Intercep
         return true;
     }
 
-    @Override
-    public Processor createProcessor(final RouteContext routeContext) throws Exception {
-        // create the detour
-        final Processor detour = this.createChildProcessor(routeContext, true);
-        final String matchURI = getUri();
-
-        // register endpoint callback so we can proxy the endpoint
-        routeContext.getCamelContext().addRegisterEndpointCallback(new EndpointStrategy() {
-            public Endpoint registerEndpoint(String uri, Endpoint endpoint) {
-                if (endpoint instanceof InterceptSendToEndpoint) {
-                    // endpoint already decorated
-                    return endpoint;
-                } else if (matchURI == null || matchPattern(routeContext.getCamelContext(), uri, matchURI)) {
-                    // only proxy if the uri is matched decorate endpoint with our proxy
-                    // should be false by default
-                    boolean skip = getSkipSendToOriginalEndpoint() != null && getSkipSendToOriginalEndpoint();
-                    InterceptSendToEndpoint proxy = new InterceptSendToEndpoint(endpoint, skip);
-                    proxy.setDetour(detour);
-                    return proxy;
-                } else {
-                    // no proxy so return regular endpoint
-                    return endpoint;
-                }
-            }
-        });
-
-
-        // remove the original intercepted route from the outputs as we do not intercept as the regular interceptor
-        // instead we use the proxy endpoints producer do the triggering. That is we trigger when someone sends
-        // an exchange to the endpoint, see InterceptSendToEndpoint for details.
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        List<ProcessorDefinition<?>> outputs = route.getOutputs();
-        outputs.remove(this);
-
-        return new InterceptEndpointProcessor(matchURI, detour);
-    }
-
-    /**
-     * Does the uri match the pattern.
-     *
-     * @param camelContext the CamelContext
-     * @param uri the uri
-     * @param pattern the pattern, which can be an endpoint uri as well
-     * @return <tt>true</tt> if matched and we should intercept, <tt>false</tt> if not matched, and not intercept.
-     */
-    protected boolean matchPattern(CamelContext camelContext, String uri, String pattern) {
-        // match using the pattern as-is
-        boolean match = EndpointHelper.matchEndpoint(camelContext, uri, pattern);
-        if (!match) {
-            try {
-                // the pattern could be an uri, so we need to normalize it before matching again
-                pattern = URISupport.normalizeUri(pattern);
-                match = EndpointHelper.matchEndpoint(camelContext, uri, pattern);
-            } catch (Exception e) {
-                // ignore
-            }
-        }
-        return match;
-    }
-
     /**
      * Applies this interceptor only if the given predicate is true
      *
diff --git a/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java b/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
index d1ca72b..09b2a07 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/LoadBalanceDefinition.java
@@ -28,7 +28,6 @@ import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.loadbalancer.CustomLoadBalancerDefinition;
 import org.apache.camel.model.loadbalancer.FailoverLoadBalancerDefinition;
 import org.apache.camel.model.loadbalancer.RandomLoadBalancerDefinition;
@@ -38,7 +37,6 @@ import org.apache.camel.model.loadbalancer.TopicLoadBalancerDefinition;
 import org.apache.camel.model.loadbalancer.WeightedLoadBalancerDefinition;
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.CollectionStringBuffer;
 
 /**
@@ -97,46 +95,6 @@ public class LoadBalanceDefinition extends ProcessorDefinition<LoadBalanceDefini
         loadBalancerType = loadbalancer;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // the load balancer is stateful so we should only create it once in case its used from a context scoped error handler
-
-        LoadBalancer loadBalancer = loadBalancerType.getLoadBalancer(routeContext);
-        if (loadBalancer == null) {
-            // then create it and reuse it
-            loadBalancer = loadBalancerType.createLoadBalancer(routeContext);
-            loadBalancerType.setLoadBalancer(loadBalancer);
-
-            // some load balancer can only support a fixed number of outputs
-            int max = loadBalancerType.getMaximumNumberOfOutputs();
-            int size = getOutputs().size();
-            if (size > max) {
-                throw new IllegalArgumentException("To many outputs configured on " + loadBalancerType + ": " + size + " > " + max);
-            }
-
-            for (ProcessorDefinition<?> processorType : getOutputs()) {
-                // output must not be another load balancer
-                // check for instanceof as the code below as there is compilation errors on earlier versions of JDK6
-                // on Windows boxes or with IBM JDKs etc.
-                if (LoadBalanceDefinition.class.isInstance(processorType)) {
-                    throw new IllegalArgumentException("Loadbalancer already configured to: " + loadBalancerType + ". Cannot set it to: " + processorType);
-                }
-                Processor processor = createProcessor(routeContext, processorType);
-                processor = wrapChannel(routeContext, processor, processorType);
-                loadBalancer.addProcessor(processor);
-            }
-        }
-
-        Boolean inherit = inheritErrorHandler;
-        if (loadBalancerType instanceof FailoverLoadBalancerDefinition) {
-            // special for failover load balancer where you can configure it to not inherit error handler for its children
-            // but the load balancer itself should inherit so Camels error handler can react afterwards
-            inherit = true;
-        }
-        Processor target = wrapChannel(routeContext, loadBalancer, this, inherit);
-        return target;
-    }
-    
     // Fluent API
     // -------------------------------------------------------------------------
 
@@ -148,7 +106,7 @@ public class LoadBalanceDefinition extends ProcessorDefinition<LoadBalanceDefini
      */
     public LoadBalanceDefinition loadBalance(LoadBalancer loadBalancer) {
         CustomLoadBalancerDefinition def = new CustomLoadBalancerDefinition();
-        def.setLoadBalancer(loadBalancer);
+        def.setCustomLoadBalancer(loadBalancer);
         setLoadBalancerType(def);
         return this;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/LoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/LoadBalancerDefinition.java
index 778a949..b20df21 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/LoadBalancerDefinition.java
@@ -27,9 +27,7 @@ import javax.xml.namespace.QName;
 
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.IntrospectionSupport;
-import org.apache.camel.util.StringHelper;
 
 /**
  * Balances message processing among a number of nodes
@@ -71,17 +69,17 @@ public class LoadBalancerDefinition extends IdentifiedType implements OtherAttri
     /**
      * Maximum number of outputs, as some load balancers only support 1 processor
      */
-    protected int getMaximumNumberOfOutputs() {
+    public int getMaximumNumberOfOutputs() {
         return Integer.MAX_VALUE;
     }
 
     /**
      * Allows derived classes to customize the load balancer
      */
-    protected void configureLoadBalancer(LoadBalancer loadBalancer) {
+    public void configureLoadBalancer(LoadBalancer loadBalancer) {
     }
 
-    public LoadBalancer getLoadBalancer(RouteContext routeContext) {
+    public LoadBalancer getLoadBalancer() {
         return loadBalancer;
     }
 
@@ -89,6 +87,10 @@ public class LoadBalancerDefinition extends IdentifiedType implements OtherAttri
         this.loadBalancer = loadBalancer;
     }
 
+    public String getLoadBalancerTypeName() {
+        return loadBalancerTypeName;
+    }
+
     @Override
     public Map<QName, Object> getOtherAttributes() {
         return otherAttributes;
@@ -99,25 +101,6 @@ public class LoadBalancerDefinition extends IdentifiedType implements OtherAttri
         this.otherAttributes = otherAttributes;
     }
 
-    /**
-     * Factory method to create the load balancer from the loadBalancerTypeName
-     */
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        StringHelper.notEmpty(loadBalancerTypeName, "loadBalancerTypeName", this);
-
-        LoadBalancer answer = null;
-        if (loadBalancerTypeName != null) {
-            Class<?> type = routeContext.getCamelContext().getClassResolver().resolveClass(loadBalancerTypeName, LoadBalancer.class);
-            if (type == null) {
-                throw new IllegalArgumentException("Cannot find class: " + loadBalancerTypeName + " in the classpath");
-            }
-            answer = (LoadBalancer) routeContext.getCamelContext().getInjector().newInstance(type);
-            configureLoadBalancer(answer);
-        }
-
-        return answer;
-    }
-
     @Override
     public String toString() {
         if (loadBalancer != null) {
diff --git a/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java b/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java
index e6b28d5..a482bf4 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java
@@ -16,27 +16,14 @@
  */
 package org.apache.camel.model;
 
-import java.util.Map;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Exchange;
-import org.apache.camel.Expression;
 import org.apache.camel.LoggingLevel;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.DefaultMaskingFormatter;
-import org.apache.camel.processor.LogProcessor;
-import org.apache.camel.spi.MaskingFormatter;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.spi.CamelLogger;
-import org.apache.camel.util.ObjectHelper;
-import org.apache.camel.util.StringHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,66 +72,6 @@ public class LogDefinition extends NoOutputDefinition<LogDefinition> {
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        StringHelper.notEmpty(message, "message", this);
-
-        // use simple language for the message string to give it more power
-        Expression exp = routeContext.getCamelContext().resolveLanguage("simple").createExpression(message);
-
-        // get logger explicitely set in the definition
-        Logger logger = this.getLogger();
-
-        // get logger which may be set in XML definition
-        if (logger == null && ObjectHelper.isNotEmpty(loggerRef)) {
-            logger = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), loggerRef, Logger.class);
-        }
-
-        if (logger == null) {
-            // first - try to lookup single instance in the registry, just like LogComponent
-            Map<String, Logger> availableLoggers = routeContext.lookupByType(Logger.class);
-            if (availableLoggers.size() == 1) {
-                logger = availableLoggers.values().iterator().next();
-                LOG.debug("Using custom Logger: {}", logger);
-            } else if (availableLoggers.size() > 1) {
-                // we should log about this somewhere...
-                LOG.debug("More than one {} instance found in the registry. Falling back to create logger by name.", Logger.class.getName());
-            }
-        }
-
-        if (logger == null) {
-            String name = getLogName();
-            if (name == null) {
-                name = routeContext.getCamelContext().getGlobalOption(Exchange.LOG_EIP_NAME);
-                if (name != null) {
-                    LOG.debug("Using logName from CamelContext properties: {}", name);
-                }
-            }
-            if (name == null) {
-                name = routeContext.getRoute().getId();
-                LOG.debug("LogName is not configured, using route id as logName: {}", name);
-            }
-            logger = LoggerFactory.getLogger(name);
-        }
-
-        // should be INFO by default
-        LoggingLevel level = getLoggingLevel() != null ? getLoggingLevel() : LoggingLevel.INFO;
-        CamelLogger camelLogger = new CamelLogger(logger, level, getMarker());
-
-        return new LogProcessor(exp, camelLogger, getMaskingFormatter(routeContext), routeContext.getCamelContext().getLogListeners());
-    }
-
-    private MaskingFormatter getMaskingFormatter(RouteContext routeContext) {
-        if (routeContext.isLogMask()) {
-            MaskingFormatter formatter = routeContext.getCamelContext().getRegistry().lookupByNameAndType(Constants.CUSTOM_LOG_MASK_REF, MaskingFormatter.class);
-            if (formatter == null) {
-                formatter = new DefaultMaskingFormatter();
-            }
-            return formatter;
-        }
-        return null;
-    }
-
-    @Override
     public void addOutput(ProcessorDefinition<?> output) {
         // add outputs on parent as this log does not support outputs
         getParent().addOutput(output);
diff --git a/camel-core/src/main/java/org/apache/camel/model/LoopDefinition.java b/camel-core/src/main/java/org/apache/camel/model/LoopDefinition.java
index 68100f9..f53c3c6 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LoopDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/LoopDefinition.java
@@ -23,11 +23,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.LoopProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Processes a message multiple times
@@ -108,22 +105,6 @@ public class LoopDefinition extends ExpressionNode {
         return "loop[" + getExpression() + "]";
     }
     
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor output = this.createChildProcessor(routeContext, true);
-        boolean isCopy = getCopy() != null && getCopy();
-        boolean isWhile = getDoWhile() != null && getDoWhile();
-
-        Predicate predicate = null;
-        Expression expression = null;
-        if (isWhile) {
-            predicate = getExpression().createPredicate(routeContext);
-        } else {
-            expression = getExpression().createExpression(routeContext);
-        }
-        return new LoopProcessor(output, expression, predicate, isCopy);
-    }
-
     /**
      * Expression to define how many times we should loop. Notice the expression is only evaluated once, and should return
      * a number as how many times to loop. A value of zero or negative means no looping. The loop is like a for-loop fashion,
diff --git a/camel-core/src/main/java/org/apache/camel/model/MarshalDefinition.java b/camel-core/src/main/java/org/apache/camel/model/MarshalDefinition.java
index 569cd66..7f4fe38 100644
--- a/camel-core/src/main/java/org/apache/camel/model/MarshalDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/MarshalDefinition.java
@@ -18,12 +18,10 @@ package org.apache.camel.model;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
 import org.apache.camel.model.dataformat.ASN1DataFormat;
 import org.apache.camel.model.dataformat.AvroDataFormat;
 import org.apache.camel.model.dataformat.Base64DataFormat;
@@ -62,10 +60,7 @@ import org.apache.camel.model.dataformat.XmlRpcDataFormat;
 import org.apache.camel.model.dataformat.YAMLDataFormat;
 import org.apache.camel.model.dataformat.ZipDataFormat;
 import org.apache.camel.model.dataformat.ZipFileDataFormat;
-import org.apache.camel.processor.MarshalProcessor;
-import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Marshals data into a specified format for transmission over a transport or component
@@ -157,9 +152,4 @@ public class MarshalDefinition extends NoOutputDefinition<MarshalDefinition> {
         this.dataFormatType = dataFormatType;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        DataFormat dataFormat = DataFormatDefinition.getDataFormat(routeContext, getDataFormatType(), null);
-        return new MarshalProcessor(dataFormat);
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/MulticastDefinition.java b/camel-core/src/main/java/org/apache/camel/model/MulticastDefinition.java
index aa19733..7c35301 100644
--- a/camel-core/src/main/java/org/apache/camel/model/MulticastDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/MulticastDefinition.java
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.model;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.ExecutorService;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -26,18 +24,11 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
 import org.apache.camel.Processor;
 import org.apache.camel.builder.AggregationStrategyClause;
 import org.apache.camel.builder.ProcessClause;
-import org.apache.camel.processor.MulticastProcessor;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
-import org.apache.camel.processor.aggregate.ShareUnitOfWorkAggregationStrategy;
-import org.apache.camel.processor.aggregate.UseLatestAggregationStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
 
 /**
  *  Routes the same message to multiple paths either sequentially or in parallel.
@@ -95,19 +86,6 @@ public class MulticastDefinition extends OutputDefinition<MulticastDefinition> i
         return "multicast";
     }
     
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor answer = this.createChildProcessor(routeContext, true);
-
-        // force the answer as a multicast processor even if there is only one child processor in the multicast
-        if (!(answer instanceof MulticastProcessor)) {
-            List<Processor> list = new ArrayList<>(1);
-            list.add(answer);
-            answer = createCompositeProcessor(routeContext, list);
-        }
-        return answer;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
@@ -326,67 +304,6 @@ public class MulticastDefinition extends OutputDefinition<MulticastDefinition> i
         return this;
     }
 
-    protected Processor createCompositeProcessor(RouteContext routeContext, List<Processor> list) throws Exception {
-        final AggregationStrategy strategy = createAggregationStrategy(routeContext);
-
-        boolean isParallelProcessing = getParallelProcessing() != null && getParallelProcessing();
-        boolean isShareUnitOfWork = getShareUnitOfWork() != null && getShareUnitOfWork();
-        boolean isStreaming = getStreaming() != null && getStreaming();
-        boolean isStopOnException = getStopOnException() != null && getStopOnException();
-        boolean isParallelAggregate = getParallelAggregate() != null && getParallelAggregate();
-        boolean isStopOnAggregateException = getStopOnAggregateException() != null && getStopOnAggregateException();
-
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, isParallelProcessing);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "Multicast", this, isParallelProcessing);
-
-        long timeout = getTimeout() != null ? getTimeout() : 0;
-        if (timeout > 0 && !isParallelProcessing) {
-            throw new IllegalArgumentException("Timeout is used but ParallelProcessing has not been enabled.");
-        }
-        if (onPrepareRef != null) {
-            onPrepare = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onPrepareRef, Processor.class);
-        }
-
-        MulticastProcessor answer = new MulticastProcessor(routeContext.getCamelContext(), list, strategy, isParallelProcessing,
-                                      threadPool, shutdownThreadPool, isStreaming, isStopOnException, timeout, onPrepare, isShareUnitOfWork, isParallelAggregate, isStopOnAggregateException);
-        return answer;
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && strategyRef != null) {
-            Object aggStrategy = routeContext.lookup(strategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getStrategyMethodName());
-                if (getStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + strategyRef);
-            }
-        }
-
-        if (strategy == null) {
-            // default to use latest aggregation strategy
-            strategy = new UseLatestAggregationStrategy();
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        if (shareUnitOfWork != null && shareUnitOfWork) {
-            // wrap strategy in share unit of work
-            strategy = new ShareUnitOfWorkAggregationStrategy(strategy);
-        }
-
-        return strategy;
-    }
-
     public AggregationStrategy getAggregationStrategy() {
         return aggregationStrategy;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java b/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
index 6ce0781..eb28c3a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/OnCompletionDefinition.java
@@ -34,11 +34,8 @@ import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
-import org.apache.camel.processor.CamelInternalProcessor;
-import org.apache.camel.processor.OnCompletionProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Route to be executed when normal route processing completes
@@ -79,6 +76,10 @@ public class OnCompletionDefinition extends ProcessorDefinition<OnCompletionDefi
         return routeScoped != null ? routeScoped : false;
     }
 
+    public Boolean getRouteScoped() {
+        return routeScoped;
+    }
+
     public Processor getOnCompletion(String routeId) {
         return onCompletions.get(routeId);
     }
@@ -87,6 +88,10 @@ public class OnCompletionDefinition extends ProcessorDefinition<OnCompletionDefi
         return onCompletions.values();
     }
 
+    public void setOnCompletion(String routeId, Processor processor) {
+        onCompletions.put(routeId, processor);
+    }
+
     @Override
     public String toString() {
         return "onCompletion[" + getOutputs() + "]";
@@ -112,57 +117,6 @@ public class OnCompletionDefinition extends ProcessorDefinition<OnCompletionDefi
         return true;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // assign whether this was a route scoped onCompletion or not
-        // we need to know this later when setting the parent, as only route scoped should have parent
-        // Note: this logic can possible be removed when the Camel routing engine decides at runtime
-        // to apply onCompletion in a more dynamic fashion than current code base
-        // and therefore is in a better position to decide among context/route scoped OnCompletion at runtime
-        if (routeScoped == null) {
-            routeScoped = super.getParent() != null;
-        }
-
-        boolean isOnCompleteOnly = getOnCompleteOnly() != null && getOnCompleteOnly();
-        boolean isOnFailureOnly = getOnFailureOnly() != null && getOnFailureOnly();
-        boolean isParallelProcessing = getParallelProcessing() != null && getParallelProcessing();
-        boolean original = getUseOriginalMessagePolicy() != null && getUseOriginalMessagePolicy();
-
-        if (isOnCompleteOnly && isOnFailureOnly) {
-            throw new IllegalArgumentException("Both onCompleteOnly and onFailureOnly cannot be true. Only one of them can be true. On node: " + this);
-        }
-        if (original) {
-            // ensure allow original is turned on
-            routeContext.setAllowUseOriginalMessage(true);
-        }
-
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        String routeId = route.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
-
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        // wrap the on completion route in a unit of work processor
-        CamelInternalProcessor internal = new CamelInternalProcessor(childProcessor);
-        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
-
-        onCompletions.put(routeId, internal);
-
-        Predicate when = null;
-        if (onWhen != null) {
-            when = onWhen.getExpression().createPredicate(routeContext);
-        }
-
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, isParallelProcessing);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "OnCompletion", this, isParallelProcessing);
-
-        // should be after consumer by default
-        boolean afterConsumer = mode == null || mode == OnCompletionMode.AfterConsumer;
-
-        OnCompletionProcessor answer = new OnCompletionProcessor(routeContext.getCamelContext(), internal,
-                threadPool, shutdownThreadPool, isOnCompleteOnly, isOnFailureOnly, when, original, afterConsumer);
-        return answer;
-    }
-
     /**
      * Removes all existing {@link org.apache.camel.model.OnCompletionDefinition} from the definition.
      * <p/>
diff --git a/camel-core/src/main/java/org/apache/camel/model/OnExceptionDefinition.java b/camel-core/src/main/java/org/apache/camel/model/OnExceptionDefinition.java
index 80e69f6..df3e8e7 100644
--- a/camel-core/src/main/java/org/apache/camel/model/OnExceptionDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/OnExceptionDefinition.java
@@ -35,16 +35,10 @@ import org.apache.camel.Expression;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
-import org.apache.camel.Route;
-import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.ExpressionBuilder;
-import org.apache.camel.processor.CatchProcessor;
-import org.apache.camel.processor.FatalFallbackErrorHandler;
 import org.apache.camel.processor.RedeliveryPolicy;
 import org.apache.camel.spi.AsPredicate;
-import org.apache.camel.spi.ClassResolver;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.CamelContextHelper;
 import org.apache.camel.support.ExpressionToPredicateAdapter;
 import org.apache.camel.util.ObjectHelper;
@@ -119,6 +113,10 @@ public class OnExceptionDefinition extends ProcessorDefinition<OnExceptionDefini
         return routeScoped != null ? routeScoped : false;
     }
 
+    public Boolean getRouteScoped() {
+        return routeScoped;
+    }
+
     @Override
     public String toString() {
         return "OnException[" + description() + " -> " + getOutputs() + "]";
@@ -175,81 +173,7 @@ public class OnExceptionDefinition extends ProcessorDefinition<OnExceptionDefini
         }
     }
 
-    public void addRoutes(RouteContext routeContext, Collection<Route> routes) throws Exception {
-        // assign whether this was a route scoped onException or not
-        // we need to know this later when setting the parent, as only route scoped should have parent
-        // Note: this logic can possible be removed when the Camel routing engine decides at runtime
-        // to apply onException in a more dynamic fashion than current code base
-        // and therefore is in a better position to decide among context/route scoped OnException at runtime
-        if (routeScoped == null) {
-            routeScoped = super.getParent() != null;
-        }
-
-        setHandledFromExpressionType(routeContext);
-        setContinuedFromExpressionType(routeContext);
-        setRetryWhileFromExpressionType(routeContext);
-        setOnRedeliveryFromRedeliveryRef(routeContext);
-        setOnExceptionOccurredFromOnExceptionOccurredRef(routeContext);
-
-        // load exception classes
-        if (exceptions != null && !exceptions.isEmpty()) {
-            exceptionClasses = createExceptionClasses(routeContext.getCamelContext().getClassResolver());
-        }
-
-        // must validate configuration before creating processor
-        validateConfiguration();
-
-        if (useOriginalMessagePolicy != null && useOriginalMessagePolicy) {
-            // ensure allow original is turned on
-            routeContext.setAllowUseOriginalMessage(true);
-        }
-
-        // lets attach this on exception to the route error handler
-        Processor child = createOutputsProcessor(routeContext);
-        if (child != null) {
-            // wrap in our special safe fallback error handler if OnException have child output
-            Processor errorHandler = new FatalFallbackErrorHandler(child);
-            String id = routeContext.getRoute().getId();
-            errorHandlers.put(id, errorHandler);
-        }
-        // lookup the error handler builder
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        ErrorHandlerBuilder builder = (ErrorHandlerBuilder) route.getErrorHandlerBuilder();
-        // and add this as error handlers
-        builder.addErrorHandlers(routeContext, this);
-    }
-
-    @Override
-    public CatchProcessor createProcessor(RouteContext routeContext) throws Exception {
-        // load exception classes
-        if (exceptions != null && !exceptions.isEmpty()) {
-            exceptionClasses = createExceptionClasses(routeContext.getCamelContext().getClassResolver());
-        }
-
-        if (useOriginalMessagePolicy != null && useOriginalMessagePolicy) {
-            // ensure allow original is turned on
-            routeContext.setAllowUseOriginalMessage(true);
-        }
-
-        // must validate configuration before creating processor
-        validateConfiguration();
-
-        Processor childProcessor = this.createChildProcessor(routeContext, false);
-
-        Predicate when = null;
-        if (onWhen != null) {
-            when = onWhen.getExpression().createPredicate(routeContext);
-        }
-
-        Predicate handle = null;
-        if (handled != null) {
-            handle = handled.createPredicate(routeContext);
-        }
-
-        return new CatchProcessor(getExceptionClasses(), childProcessor, when, handle);
-    }
-
-    protected void validateConfiguration() {
+    public void validateConfiguration() {
         if (isInheritErrorHandler() != null && isInheritErrorHandler()) {
             throw new IllegalArgumentException(this + " cannot have the inheritErrorHandler option set to true");
         }
@@ -871,6 +795,10 @@ public class OnExceptionDefinition extends ProcessorDefinition<OnExceptionDefini
         return errorHandlers.values();
     }
 
+    public void setErrorHandler(String routeId, Processor errorHandler) {
+        errorHandlers.put(routeId, errorHandler);
+    }
+
     public RedeliveryPolicyDefinition getRedeliveryPolicy() {
         return redeliveryPolicyType;
     }
@@ -1008,50 +936,4 @@ public class OnExceptionDefinition extends ProcessorDefinition<OnExceptionDefini
         return redeliveryPolicyType;
     }
 
-    protected List<Class<? extends Throwable>> createExceptionClasses(ClassResolver resolver) throws ClassNotFoundException {
-        List<String> list = getExceptions();
-        List<Class<? extends Throwable>> answer = new ArrayList<>(list.size());
-        for (String name : list) {
-            Class<? extends Throwable> type = resolver.resolveMandatoryClass(name, Throwable.class);
-            answer.add(type);
-        }
-        return answer;
-    }
-
-    private void setHandledFromExpressionType(RouteContext routeContext) {
-        if (getHandled() != null && handledPolicy == null && routeContext != null) {
-            handled(getHandled().createPredicate(routeContext));
-        }
-    }
-
-    private void setContinuedFromExpressionType(RouteContext routeContext) {
-        if (getContinued() != null && continuedPolicy == null && routeContext != null) {
-            continued(getContinued().createPredicate(routeContext));
-        }
-    }
-
-    private void setRetryWhileFromExpressionType(RouteContext routeContext) {
-        if (getRetryWhile() != null && retryWhilePolicy == null && routeContext != null) {
-            retryWhile(getRetryWhile().createPredicate(routeContext));
-        }
-    }
-
-    private void setOnRedeliveryFromRedeliveryRef(RouteContext routeContext) {
-        // lookup onRedelivery if ref is provided
-        if (ObjectHelper.isNotEmpty(onRedeliveryRef)) {
-            // if ref is provided then use mandatory lookup to fail if not found
-            Processor onRedelivery = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onRedeliveryRef, Processor.class);
-            setOnRedelivery(onRedelivery);
-        }
-    }
-
-    private void setOnExceptionOccurredFromOnExceptionOccurredRef(RouteContext routeContext) {
-        // lookup onRedelivery if ref is provided
-        if (ObjectHelper.isNotEmpty(onExceptionOccurredRef)) {
-            // if ref is provided then use mandatory lookup to fail if not found
-            Processor onExceptionOccurred = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onExceptionOccurredRef, Processor.class);
-            setOnExceptionOccurred(onExceptionOccurred);
-        }
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/OnFallbackDefinition.java b/camel-core/src/main/java/org/apache/camel/model/OnFallbackDefinition.java
index 637b6c6..5d460fd 100644
--- a/camel-core/src/main/java/org/apache/camel/model/OnFallbackDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/OnFallbackDefinition.java
@@ -23,9 +23,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.CollectionStringBuffer;
 
 /**
@@ -53,11 +51,6 @@ public class OnFallbackDefinition extends OutputDefinition<OnFallbackDefinition>
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return this.createChildProcessor(routeContext, false);
-    }
-
-    @Override
     public String getShortName() {
         return "onFallback";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/OtherwiseDefinition.java b/camel-core/src/main/java/org/apache/camel/model/OtherwiseDefinition.java
index 3866242..e03f7a2 100644
--- a/camel-core/src/main/java/org/apache/camel/model/OtherwiseDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/OtherwiseDefinition.java
@@ -22,9 +22,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.CollectionStringBuffer;
 
 /**
@@ -44,11 +42,6 @@ public class OtherwiseDefinition extends OutputDefinition<OtherwiseDefinition> {
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return this.createChildProcessor(routeContext, false);
-    }
-
-    @Override
     public String getShortName() {
         return "otherwise";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java b/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
index b0a102a..1b6efd2 100644
--- a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
@@ -20,9 +20,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Routes the message to a sequence of processors.
@@ -45,7 +43,4 @@ public class PipelineDefinition extends OutputDefinition<PipelineDefinition> {
         return "pipeline";
     }
 
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return this.createChildProcessor(routeContext, true);
-    }
 }
\ No newline at end of file
diff --git a/camel-core/src/main/java/org/apache/camel/model/PolicyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/PolicyDefinition.java
index df5a63a..edc6e5a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/PolicyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/PolicyDefinition.java
@@ -22,14 +22,9 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.Service;
-import org.apache.camel.processor.WrapProcessor;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Policy;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.TransactedPolicy;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Defines a policy the route will use
@@ -61,6 +56,14 @@ public class PolicyDefinition extends OutputDefinition<PolicyDefinition> {
         return "Policy[" + description() + "]";
     }
     
+    public Policy getPolicy() {
+        return policy;
+    }
+
+    public Class<? extends Policy> getType() {
+        return type;
+    }
+
     protected String description() {
         if (policy != null) {
             return policy.toString();
@@ -133,33 +136,4 @@ public class PolicyDefinition extends OutputDefinition<PolicyDefinition> {
         return this;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Policy policy = resolvePolicy(routeContext);
-        ObjectHelper.notNull(policy, "policy", this);
-
-        // before wrap
-        policy.beforeWrap(routeContext, this);
-
-        // create processor after the before wrap
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        // wrap
-        Processor target = policy.wrap(routeContext, childProcessor);
-
-        if (!(target instanceof Service)) {
-            // wrap the target so it becomes a service and we can manage its lifecycle
-            target = new WrapProcessor(target, childProcessor);
-        }
-        return target;
-    }
-
-    protected Policy resolvePolicy(RouteContext routeContext) {
-        if (policy != null) {
-            return policy;
-        }
-        // reuse code on transacted definition to do the resolution
-        return TransactedDefinition.doResolvePolicy(routeContext, getRef(), type);
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/PollEnrichDefinition.java b/camel-core/src/main/java/org/apache/camel/model/PollEnrichDefinition.java
index 87647d0..e1d375d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/PollEnrichDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/PollEnrichDefinition.java
@@ -22,16 +22,10 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
-import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.impl.DefaultConsumerCache;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.PollEnricher;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Enriches messages with data polled from a secondary resource
@@ -82,58 +76,6 @@ public class PollEnrichDefinition extends NoOutputExpressionNode {
         return "pollEnrich[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-
-        // if no timeout then we should block, and there use a negative timeout
-        long time = timeout != null ? timeout : -1;
-        boolean isIgnoreInvalidEndpoint = getIgnoreInvalidEndpoint() != null && getIgnoreInvalidEndpoint();
-        Expression exp = getExpression().createExpression(routeContext);
-
-        PollEnricher enricher = new PollEnricher(exp, time);
-
-        AggregationStrategy strategy = createAggregationStrategy(routeContext);
-        if (strategy == null) {
-            enricher.setDefaultAggregationStrategy();
-        } else {
-            enricher.setAggregationStrategy(strategy);
-        }
-        if (getAggregateOnException() != null) {
-            enricher.setAggregateOnException(getAggregateOnException());
-        }
-        if (getCacheSize() != null) {
-            enricher.setCacheSize(getCacheSize());
-        }
-        enricher.setIgnoreInvalidEndpoint(isIgnoreInvalidEndpoint);
-
-        return enricher;
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && aggregationStrategyRef != null) {
-            Object aggStrategy = routeContext.lookup(aggregationStrategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getAggregationStrategyMethodName());
-                if (getAggregationStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getAggregationStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getAggregationStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + aggregationStrategyRef);
-            }
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        return strategy;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessDefinition.java
index 31c6da3..4e2829d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessDefinition.java
@@ -22,13 +22,8 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.AsyncProcessor;
 import org.apache.camel.Processor;
-import org.apache.camel.Service;
-import org.apache.camel.processor.DelegateAsyncProcessor;
-import org.apache.camel.processor.DelegateSyncProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.ObjectHelper;
 
 /**
@@ -79,6 +74,10 @@ public class ProcessDefinition extends NoOutputDefinition<ProcessDefinition> {
         }
     }
 
+    public Processor getProcessor() {
+        return processor;
+    }
+
     public String getRef() {
         return ref;
     }
@@ -90,25 +89,4 @@ public class ProcessDefinition extends NoOutputDefinition<ProcessDefinition> {
         this.ref = ref;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        Processor answer = processor;
-        if (processor == null) {
-            ObjectHelper.notNull(ref, "ref", this);
-            answer = routeContext.mandatoryLookup(getRef(), Processor.class);
-        }
-
-        // ensure its wrapped in a Service so we can manage it from eg. JMX
-        // (a Processor must be a Service to be enlisted in JMX)
-        if (!(answer instanceof Service)) {
-            if (answer instanceof AsyncProcessor) {
-                // the processor is async by nature so use the async delegate
-                answer = new DelegateAsyncProcessor(answer);
-            } else {
-                // the processor is sync by nature so use the sync delegate
-                answer = new DelegateSyncProcessor(answer);
-            }
-        }
-        return answer;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index 9094494..4e5eda6 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -36,16 +36,13 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.namespace.QName;
 
-import org.apache.camel.Channel;
 import org.apache.camel.Endpoint;
-import org.apache.camel.ErrorHandlerFactory;
 import org.apache.camel.Exchange;
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Expression;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
-import org.apache.camel.Route;
 import org.apache.camel.builder.DataFormatClause;
 import org.apache.camel.builder.EnrichClause;
 import org.apache.camel.builder.ExpressionBuilder;
@@ -58,21 +55,14 @@ import org.apache.camel.model.language.ConstantExpression;
 import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.model.language.LanguageExpression;
 import org.apache.camel.model.rest.RestDefinition;
-import org.apache.camel.processor.InterceptEndpointProcessor;
-import org.apache.camel.processor.Pipeline;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.interceptor.DefaultChannel;
-import org.apache.camel.processor.interceptor.HandleFault;
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.AsEndpointUri;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.DataFormat;
-import org.apache.camel.spi.IdAware;
 import org.apache.camel.spi.IdempotentRepository;
 import org.apache.camel.spi.InterceptStrategy;
-import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.spi.Policy;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.ExpressionAdapter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -164,47 +154,6 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
         return false;
     }
 
-    /**
-     * Override this in definition class and implement logic to create the processor
-     * based on the definition model.
-     */
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        throw new UnsupportedOperationException("Not implemented yet for class: " + getClass().getName());
-    }
-
-    /**
-     * Prefer to use {#link #createChildProcessor}.
-     */
-    public Processor createOutputsProcessor(RouteContext routeContext) throws Exception {
-        Collection<ProcessorDefinition<?>> outputs = getOutputs();
-        return createOutputsProcessor(routeContext, outputs);
-    }
-
-    /**
-     * Creates the child processor (outputs) from the current definition
-     *
-     * @param routeContext   the route context
-     * @param mandatory      whether or not children is mandatory (ie the definition should have outputs)
-     * @return the created children, or <tt>null</tt> if definition had no output
-     * @throws Exception is thrown if error creating the child or if it was mandatory and there was no output defined on definition
-     */
-    public Processor createChildProcessor(RouteContext routeContext, boolean mandatory) throws Exception {
-        Processor children = null;
-        // at first use custom factory
-        if (routeContext.getCamelContext().getProcessorFactory() != null) {
-            children = routeContext.getCamelContext().getProcessorFactory().createChildProcessor(routeContext, this, mandatory);
-        }
-        // fallback to default implementation if factory did not create the child
-        if (children == null) {
-            children = createOutputsProcessor(routeContext);
-        }
-
-        if (children == null && mandatory) {
-            throw new IllegalArgumentException("Definition has no children on " + this);
-        }
-        return children;
-    }
-
     @Override
     public void addOutput(ProcessorDefinition<?> output) {
         if (!blocks.isEmpty()) {
@@ -230,335 +179,10 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
         blocks.clear();
     }
 
-    public void addRoutes(RouteContext routeContext, Collection<Route> routes) throws Exception {
-        Processor processor = makeProcessor(routeContext);
-        if (processor == null) {
-            // no processor to add
-            return;
-        }
-
-        if (!routeContext.isRouteAdded()) {
-            boolean endpointInterceptor = false;
-
-            // are we routing to an endpoint interceptor, if so we should not add it as an event driven
-            // processor as we use the producer to trigger the interceptor
-            if (processor instanceof Channel) {
-                Channel channel = (Channel) processor;
-                Processor next = channel.getNextProcessor();
-                if (next instanceof InterceptEndpointProcessor) {
-                    endpointInterceptor = true;
-                }
-            }
-
-            // only add regular processors as event driven
-            if (endpointInterceptor) {
-                log.debug("Endpoint interceptor should not be added as an event driven consumer route: {}", processor);
-            } else {
-                log.trace("Adding event driven processor: {}", processor);
-                routeContext.addEventDrivenProcessor(processor);
-            }
-
-        }
-    }
-
-    /**
-     * Wraps the child processor in whatever necessary interceptors and error handlers
-     */
-    public Processor wrapProcessor(RouteContext routeContext, Processor processor) throws Exception {
-        // dont double wrap
-        if (processor instanceof Channel) {
-            return processor;
-        }
-        return wrapChannel(routeContext, processor, null);
-    }
-
-    protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child) throws Exception {
-        return wrapChannel(routeContext, processor, child, isInheritErrorHandler());
-    }
-
-    protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child, Boolean inheritErrorHandler) throws Exception {
-        // put a channel in between this and each output to control the route flow logic
-        ModelChannel channel = createChannel(routeContext);
-        channel.setNextProcessor(processor);
-
-        // add interceptor strategies to the channel must be in this order: camel context, route context, local
-        addInterceptStrategies(routeContext, channel, routeContext.getCamelContext().getInterceptStrategies());
-        addInterceptStrategies(routeContext, channel, routeContext.getInterceptStrategies());
-        addInterceptStrategies(routeContext, channel, this.getInterceptStrategies());
-
-        // must do this ugly cast to avoid compiler error on AIX/HP-UX
-        ProcessorDefinition<?> defn = (ProcessorDefinition<?>) this;
-
-        // set the child before init the channel
-        channel.setChildDefinition(child);
-        channel.initChannel(defn, routeContext);
-
-        // set the error handler, must be done after init as we can set the error handler as first in the chain
-        if (defn instanceof TryDefinition || defn instanceof CatchDefinition || defn instanceof FinallyDefinition) {
-            // do not use error handler for try .. catch .. finally blocks as it will handle errors itself
-            log.trace("{} is part of doTry .. doCatch .. doFinally so no error handler is applied", defn);
-        } else if (ProcessorDefinitionHelper.isParentOfType(TryDefinition.class, defn, true)
-                || ProcessorDefinitionHelper.isParentOfType(CatchDefinition.class, defn, true)
-                || ProcessorDefinitionHelper.isParentOfType(FinallyDefinition.class, defn, true)) {
-            // do not use error handler for try .. catch .. finally blocks as it will handle errors itself
-            // by checking that any of our parent(s) is not a try .. catch or finally type
-            log.trace("{} is part of doTry .. doCatch .. doFinally so no error handler is applied", defn);
-        } else if (defn instanceof OnExceptionDefinition || ProcessorDefinitionHelper.isParentOfType(OnExceptionDefinition.class, defn, true)) {
-            log.trace("{} is part of OnException so no error handler is applied", defn);
-            // do not use error handler for onExceptions blocks as it will handle errors itself
-        } else if (defn instanceof HystrixDefinition || ProcessorDefinitionHelper.isParentOfType(HystrixDefinition.class, defn, true)) {
-            // do not use error handler for hystrix as it offers circuit breaking with fallback for its outputs
-            // however if inherit error handler is enabled, we need to wrap an error handler on the hystrix parent
-            if (inheritErrorHandler != null && inheritErrorHandler && child == null) {
-                // only wrap the parent (not the children of the hystrix)
-                wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
-            } else {
-                log.trace("{} is part of HystrixCircuitBreaker so no error handler is applied", defn);
-            }
-        } else if (defn instanceof MulticastDefinition) {
-            // do not use error handler for multicast as it offers fine grained error handlers for its outputs
-            // however if share unit of work is enabled, we need to wrap an error handler on the multicast parent
-            MulticastDefinition def = (MulticastDefinition) defn;
-            boolean isShareUnitOfWork = def.getShareUnitOfWork() != null && def.getShareUnitOfWork();
-            if (isShareUnitOfWork && child == null) {
-                // only wrap the parent (not the children of the multicast)
-                wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
-            } else {
-                log.trace("{} is part of multicast which have special error handling so no error handler is applied", defn);
-            }
-        } else {
-            // use error handler by default or if configured to do so
-            wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
-        }
-
-        // do post init at the end
-        channel.postInitChannel(defn, routeContext);
-        log.trace("{} wrapped in Channel: {}", defn, channel);
-
-        return channel;
-    }
-
-    /**
-     * Wraps the given channel in error handler (if error handler is inherited)
-     *
-     * @param channel             the channel
-     * @param routeContext        the route context
-     * @param inheritErrorHandler whether to inherit error handler
-     * @throws Exception can be thrown if failed to create error handler builder
-     */
-    private void wrapChannelInErrorHandler(Channel channel, RouteContext routeContext, Boolean inheritErrorHandler) throws Exception {
-        if (inheritErrorHandler == null || inheritErrorHandler) {
-            log.trace("{} is configured to inheritErrorHandler", this);
-            Processor output = channel.getOutput();
-            Processor errorHandler = wrapInErrorHandler(routeContext, output);
-            // set error handler on channel
-            channel.setErrorHandler(errorHandler);
-        } else {
-            log.debug("{} is configured to not inheritErrorHandler.", this);
-        }
-    }
-
-    /**
-     * Wraps the given output in an error handler
-     *
-     * @param routeContext the route context
-     * @param output the output
-     * @return the output wrapped with the error handler
-     * @throws Exception can be thrown if failed to create error handler builder
-     */
-    protected Processor wrapInErrorHandler(RouteContext routeContext, Processor output) throws Exception {
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        ErrorHandlerFactory builder = route.getErrorHandlerBuilder();
-        // create error handler
-        Processor errorHandler = builder.createErrorHandler(routeContext, output);
-
-        // invoke lifecycles so we can manage this error handler builder
-        for (LifecycleStrategy strategy : routeContext.getCamelContext().getLifecycleStrategies()) {
-            strategy.onErrorHandlerAdd(routeContext, errorHandler, builder);
-        }
-
-        return errorHandler;
-    }
-
-    /**
-     * Adds the given list of interceptors to the channel.
-     *
-     * @param routeContext  the route context
-     * @param channel       the channel to add strategies
-     * @param strategies    list of strategies to add.
-     */
-    protected void addInterceptStrategies(RouteContext routeContext, Channel channel, List<InterceptStrategy> strategies) {
-        for (InterceptStrategy strategy : strategies) {
-            if (!routeContext.isHandleFault() && strategy instanceof HandleFault) {
-                // handle fault is disabled so we should not add it
-                continue;
-            }
-
-            // add strategy
-            channel.addInterceptStrategy(strategy);
-        }
-    }
-
-    /**
-     * Creates a new instance of some kind of composite processor which defaults
-     * to using a {@link Pipeline} but derived classes could change the behaviour
-     */
-    protected Processor createCompositeProcessor(RouteContext routeContext, List<Processor> list) throws Exception {
-        return Pipeline.newInstance(routeContext.getCamelContext(), list);
-    }
-
-    /**
-     * Creates a new instance of the {@link Channel}.
-     */
-    protected ModelChannel createChannel(RouteContext routeContext) throws Exception {
-        return new DefaultChannel();
-    }
-
-    protected Processor createOutputsProcessor(RouteContext routeContext, Collection<ProcessorDefinition<?>> outputs) throws Exception {
-        // We will save list of actions to restore the outputs back to the original state.
-        Runnable propertyPlaceholdersChangeReverter = ProcessorDefinitionHelper.createPropertyPlaceholdersChangeReverter();
-        try {
-            return createOutputsProcessorImpl(routeContext, outputs);
-        } finally {
-            propertyPlaceholdersChangeReverter.run();
-        }
-    }
-
-    protected Processor createOutputsProcessorImpl(RouteContext routeContext, Collection<ProcessorDefinition<?>> outputs) throws Exception {
-        List<Processor> list = new ArrayList<>();
-        for (ProcessorDefinition<?> output : outputs) {
-
-            // allow any custom logic before we create the processor
-            output.preCreateProcessor();
-
-            // resolve properties before we create the processor
-            ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), output);
-
-            // resolve constant fields (eg Exchange.FILE_NAME)
-            ProcessorDefinitionHelper.resolveKnownConstantFields(output);
-
-            // also resolve properties and constant fields on embedded expressions
-            ProcessorDefinition<?> me = (ProcessorDefinition<?>) output;
-            if (me instanceof ExpressionNode) {
-                ExpressionNode exp = (ExpressionNode) me;
-                ExpressionDefinition expressionDefinition = exp.getExpression();
-                if (expressionDefinition != null) {
-                    // resolve properties before we create the processor
-                    ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
-
-                    // resolve constant fields (eg Exchange.FILE_NAME)
-                    ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
-                }
-            }
-
-            Processor processor = createProcessor(routeContext, output);
-
-            // inject id
-            if (processor instanceof IdAware) {
-                String id = output.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
-                ((IdAware) processor).setId(id);
-            }
-
-            if (output instanceof Channel && processor == null) {
-                continue;
-            }
-
-            Processor channel = wrapChannel(routeContext, processor, output);
-            list.add(channel);
-        }
-
-        // if more than one output wrap than in a composite processor else just keep it as is
-        Processor processor = null;
-        if (!list.isEmpty()) {
-            if (list.size() == 1) {
-                processor = list.get(0);
-            } else {
-                processor = createCompositeProcessor(routeContext, list);
-            }
-        }
-
-        return processor;
-    }
-
-    protected Processor createProcessor(RouteContext routeContext, ProcessorDefinition<?> output) throws Exception {
-        Processor processor = null;
-        // at first use custom factory
-        if (routeContext.getCamelContext().getProcessorFactory() != null) {
-            processor = routeContext.getCamelContext().getProcessorFactory().createProcessor(routeContext, output);
-        }
-        // fallback to default implementation if factory did not create the processor
-        if (processor == null) {
-            processor = output.createProcessor(routeContext);
-        }
-        return processor;
-    }
-
-    /**
-     * Creates the processor and wraps it in any necessary interceptors and error handlers
-     */
-    protected Processor makeProcessor(RouteContext routeContext) throws Exception {
-        // We will save list of actions to restore the definition back to the original state.
-        Runnable propertyPlaceholdersChangeReverter = ProcessorDefinitionHelper.createPropertyPlaceholdersChangeReverter();
-        try {
-            return makeProcessorImpl(routeContext);
-        } finally {
-            // Lets restore
-            propertyPlaceholdersChangeReverter.run();
-        }
-    }
-
-    private Processor makeProcessorImpl(RouteContext routeContext) throws Exception {
-        Processor processor = null;
-
-        // allow any custom logic before we create the processor
-        preCreateProcessor();
-
-        // resolve properties before we create the processor
-        ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), this);
-
-        // resolve constant fields (eg Exchange.FILE_NAME)
-        ProcessorDefinitionHelper.resolveKnownConstantFields(this);
-
-        // also resolve properties and constant fields on embedded expressions
-        ProcessorDefinition<?> me = (ProcessorDefinition<?>) this;
-        if (me instanceof ExpressionNode) {
-            ExpressionNode exp = (ExpressionNode) me;
-            ExpressionDefinition expressionDefinition = exp.getExpression();
-            if (expressionDefinition != null) {
-                // resolve properties before we create the processor
-                ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
-
-                // resolve constant fields (eg Exchange.FILE_NAME)
-                ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
-            }
-        }
-
-        // at first use custom factory
-        if (routeContext.getCamelContext().getProcessorFactory() != null) {
-            processor = routeContext.getCamelContext().getProcessorFactory().createProcessor(routeContext, this);
-        }
-        // fallback to default implementation if factory did not create the processor
-        if (processor == null) {
-            processor = createProcessor(routeContext);
-        }
-
-        // inject id
-        if (processor instanceof IdAware) {
-            String id = this.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
-            ((IdAware) processor).setId(id);
-        }
-
-        if (processor == null) {
-            // no processor to make
-            return null;
-        }
-        return wrapProcessor(routeContext, processor);
-    }
-
     /**
      * Strategy to execute any custom logic before the {@link Processor} is created.
      */
-    protected void preCreateProcessor() {
+    public void preCreateProcessor() {
         // noop
     }
 
@@ -1155,7 +779,7 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
 
     /**
      * <a href="http://camel.apache.org/pipes-nd-filters.html">Pipes and Filters EIP:</a>
-     * Creates a {@link Pipeline} so that the message
+     * Creates a {@link org.apache.camel.processor.Pipeline} so that the message
      * will get processed by each endpoint in turn and for request/response the
      * output of one endpoint will be the input of the next endpoint
      *
diff --git a/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
index 48bc007..99ad81f 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RecipientListDefinition.java
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.model;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.ExecutorService;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -26,21 +24,12 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
 import org.apache.camel.Expression;
 import org.apache.camel.Processor;
 import org.apache.camel.builder.ProcessClause;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.EvaluateExpressionProcessor;
-import org.apache.camel.processor.Pipeline;
-import org.apache.camel.processor.RecipientList;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
-import org.apache.camel.processor.aggregate.ShareUnitOfWorkAggregationStrategy;
-import org.apache.camel.processor.aggregate.UseLatestAggregationStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
 
 /**
  * Routes messages to a number of dynamically specified recipients (dynamic to)
@@ -112,114 +101,6 @@ public class RecipientListDefinition<Type extends ProcessorDefinition<Type>> ext
         return "recipientList[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        final Expression expression = getExpression().createExpression(routeContext);
-
-        boolean isParallelProcessing = getParallelProcessing() != null && getParallelProcessing();
-        boolean isStreaming = getStreaming() != null && getStreaming();
-        boolean isParallelAggregate = getParallelAggregate() != null && getParallelAggregate();
-        boolean isShareUnitOfWork = getShareUnitOfWork() != null && getShareUnitOfWork();
-        boolean isStopOnException = getStopOnException() != null && getStopOnException();
-        boolean isIgnoreInvalidEndpoints = getIgnoreInvalidEndpoints() != null && getIgnoreInvalidEndpoints();
-        boolean isStopOnAggregateException = getStopOnAggregateException() != null && getStopOnAggregateException();
-
-        RecipientList answer;
-        if (delimiter != null) {
-            answer = new RecipientList(routeContext.getCamelContext(), expression, delimiter);
-        } else {
-            answer = new RecipientList(routeContext.getCamelContext(), expression);
-        }
-        answer.setAggregationStrategy(createAggregationStrategy(routeContext));
-        answer.setParallelProcessing(isParallelProcessing);
-        answer.setParallelAggregate(isParallelAggregate);
-        answer.setStreaming(isStreaming);
-        answer.setShareUnitOfWork(isShareUnitOfWork);
-        answer.setStopOnException(isStopOnException);
-        answer.setIgnoreInvalidEndpoints(isIgnoreInvalidEndpoints);
-        answer.setStopOnAggregateException(isStopOnAggregateException);
-        if (getCacheSize() != null) {
-            answer.setCacheSize(getCacheSize());
-        }
-        if (onPrepareRef != null) {
-            onPrepare = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onPrepareRef, Processor.class);
-        }
-        if (onPrepare != null) {
-            answer.setOnPrepare(onPrepare);
-        }
-        if (getTimeout() != null) {
-            answer.setTimeout(getTimeout());
-        }
-
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, isParallelProcessing);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "RecipientList", this, isParallelProcessing);
-        answer.setExecutorService(threadPool);
-        answer.setShutdownExecutorService(shutdownThreadPool);
-        long timeout = getTimeout() != null ? getTimeout() : 0;
-        if (timeout > 0 && !isParallelProcessing) {
-            throw new IllegalArgumentException("Timeout is used but ParallelProcessing has not been enabled.");
-        }
-
-        // create a pipeline with two processors
-        // the first is the eval processor which evaluates the expression to use
-        // the second is the recipient list
-        List<Processor> pipe = new ArrayList<>(2);
-
-        // the eval processor must be wrapped in error handler, so in case there was an
-        // error during evaluation, the error handler can deal with it
-        // the recipient list is not in error handler, as its has its own special error handling
-        // when sending to the recipients individually
-        Processor evalProcessor = new EvaluateExpressionProcessor(expression);
-        evalProcessor = super.wrapInErrorHandler(routeContext, evalProcessor);
-
-        pipe.add(evalProcessor);
-        pipe.add(answer);
-
-        // wrap in nested pipeline so this appears as one processor
-        // (threads definition does this as well)
-        return new Pipeline(routeContext.getCamelContext(), pipe) {
-            @Override
-            public String toString() {
-                return "RecipientList[" + expression + "]";
-            }
-        };
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && strategyRef != null) {
-            Object aggStrategy = routeContext.lookup(strategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getStrategyMethodName());
-                if (getStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + strategyRef);
-            }
-        }
-
-        if (strategy == null) {
-            // default to use latest aggregation strategy
-            strategy = new UseLatestAggregationStrategy();
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        if (shareUnitOfWork != null && shareUnitOfWork) {
-            // wrap strategy in share unit of work
-            strategy = new ShareUnitOfWorkAggregationStrategy(strategy);
-        }
-
-        return strategy;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/RemoveHeaderDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RemoveHeaderDefinition.java
index a141983..834a3d9 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RemoveHeaderDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RemoveHeaderDefinition.java
@@ -21,11 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.RemoveHeaderProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Removes a named header from the message
@@ -59,12 +55,6 @@ public class RemoveHeaderDefinition extends NoOutputDefinition<RemoveHeaderDefin
         return "removeHeader[" + getHeaderName() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(getHeaderName(), "headerName", this);
-        return new RemoveHeaderProcessor(getHeaderName());
-    }
-
     /**
      * Name of header to remove
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/RemoveHeadersDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RemoveHeadersDefinition.java
index 673eb5c..76042a0 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RemoveHeadersDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RemoveHeadersDefinition.java
@@ -22,11 +22,7 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.RemoveHeadersProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Removes message headers whose name matches a specified pattern
@@ -70,18 +66,6 @@ public class RemoveHeadersDefinition extends NoOutputDefinition<RemoveHeadersDef
         return "removeHeaders[" + getPattern() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(getPattern(), "patterns", this);
-        if (getExcludePatterns() != null) {
-            return new RemoveHeadersProcessor(getPattern(), getExcludePatterns());
-        } else if (getExcludePattern() != null) {
-            return new RemoveHeadersProcessor(getPattern(), new String[]{getExcludePattern()});
-        } else {
-            return new RemoveHeadersProcessor(getPattern(), null);
-        }
-    }
-
     /**
      * Name or pattern of headers to remove
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/RemovePropertiesDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RemovePropertiesDefinition.java
index f1f68e4..acbc733 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RemovePropertiesDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RemovePropertiesDefinition.java
@@ -22,11 +22,7 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.RemovePropertiesProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Removes message exchange properties whose name matches a specified pattern
@@ -70,18 +66,6 @@ public class RemovePropertiesDefinition extends NoOutputDefinition<RemovePropert
         return "removeProperties[" + getPattern() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(getPattern(), "patterns", this);
-        if (getExcludePatterns() != null) {
-            return new RemovePropertiesProcessor(getPattern(), getExcludePatterns());
-        } else if (getExcludePattern() != null) {
-            return new RemovePropertiesProcessor(getPattern(), new String[]{getExcludePattern()});
-        } else {
-            return new RemovePropertiesProcessor(getPattern(), null);
-        }
-    }
-
     /**
      * Name or pattern of properties to remove
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/RemovePropertyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RemovePropertyDefinition.java
index b5bf77e..6441f4e 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RemovePropertyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RemovePropertyDefinition.java
@@ -21,11 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.RemovePropertyProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Removes a named property from the message exchange
@@ -59,12 +55,6 @@ public class RemovePropertyDefinition extends NoOutputDefinition<RemovePropertyD
         return "removeProperty[" + getPropertyName() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(getPropertyName(), "propertyName", this);
-        return new RemovePropertyProcessor(getPropertyName());
-    }
-
     /**
      * Name of property to remove
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/ResequenceDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ResequenceDefinition.java
index 921f79a..750b2d8 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ResequenceDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ResequenceDefinition.java
@@ -28,19 +28,12 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.config.BatchResequencerConfig;
 import org.apache.camel.model.config.ResequencerConfig;
 import org.apache.camel.model.config.StreamResequencerConfig;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.CamelInternalProcessor;
-import org.apache.camel.processor.Resequencer;
-import org.apache.camel.processor.StreamResequencer;
 import org.apache.camel.processor.resequencer.ExpressionResultComparator;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Resequences (re-order) messages based on an expression
@@ -338,100 +331,4 @@ public class ResequenceDefinition extends ProcessorDefinition<ResequenceDefiniti
         this.expression = expression;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // if configured from XML then streamConfig has been set with the configuration
-        if (resequencerConfig != null) {
-            if (resequencerConfig instanceof StreamResequencerConfig) {
-                streamConfig = (StreamResequencerConfig) resequencerConfig;
-            } else {
-                batchConfig = (BatchResequencerConfig) resequencerConfig;
-            }
-        }
-
-        if (streamConfig != null) {
-            return createStreamResequencer(routeContext, streamConfig);
-        } else {
-            if (batchConfig == null) {
-                // default as batch mode
-                batch();
-            }
-            return createBatchResequencer(routeContext, batchConfig);
-        }
-    }
-
-    /**
-     * Creates a batch {@link Resequencer} instance applying the given <code>config</code>.
-     * 
-     * @param routeContext route context.
-     * @param config batch resequencer configuration.
-     * @return the configured batch resequencer.
-     * @throws Exception can be thrown
-     */
-    protected Resequencer createBatchResequencer(RouteContext routeContext,
-                                                 BatchResequencerConfig config) throws Exception {
-        Processor processor = this.createChildProcessor(routeContext, true);
-        Expression expression = getExpression().createExpression(routeContext);
-
-        // and wrap in unit of work
-        CamelInternalProcessor internal = new CamelInternalProcessor(processor);
-        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
-
-        ObjectHelper.notNull(config, "config", this);
-        ObjectHelper.notNull(expression, "expression", this);
-
-        boolean isReverse = config.getReverse() != null && config.getReverse();
-        boolean isAllowDuplicates = config.getAllowDuplicates() != null && config.getAllowDuplicates();
-
-        Resequencer resequencer = new Resequencer(routeContext.getCamelContext(), internal, expression, isAllowDuplicates, isReverse);
-        resequencer.setBatchSize(config.getBatchSize());
-        resequencer.setBatchTimeout(config.getBatchTimeout());
-        resequencer.setReverse(isReverse);
-        resequencer.setAllowDuplicates(isAllowDuplicates);
-        if (config.getIgnoreInvalidExchanges() != null) {
-            resequencer.setIgnoreInvalidExchanges(config.getIgnoreInvalidExchanges());
-        }
-        return resequencer;
-    }
-
-    /**
-     * Creates a {@link StreamResequencer} instance applying the given <code>config</code>.
-     * 
-     * @param routeContext route context.
-     * @param config stream resequencer configuration.
-     * @return the configured stream resequencer.
-     * @throws Exception can be thrwon
-     */
-    protected StreamResequencer createStreamResequencer(RouteContext routeContext,
-                                                        StreamResequencerConfig config) throws Exception {
-        Processor processor = this.createChildProcessor(routeContext, true);
-        Expression expression = getExpression().createExpression(routeContext);
-
-        CamelInternalProcessor internal = new CamelInternalProcessor(processor);
-        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
-
-        ObjectHelper.notNull(config, "config", this);
-        ObjectHelper.notNull(expression, "expression", this);
-
-        ExpressionResultComparator comparator;
-        if (config.getComparatorRef() != null) {
-            comparator = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), config.getComparatorRef(), ExpressionResultComparator.class);
-        } else {
-            comparator = config.getComparator();
-        }
-        comparator.setExpression(expression);
-
-        StreamResequencer resequencer = new StreamResequencer(routeContext.getCamelContext(), internal, comparator, expression);
-        resequencer.setTimeout(config.getTimeout());
-        if (config.getDeliveryAttemptInterval() != null) {
-            resequencer.setDeliveryAttemptInterval(config.getDeliveryAttemptInterval());
-        }
-        resequencer.setCapacity(config.getCapacity());
-        resequencer.setRejectOld(config.getRejectOld());
-        if (config.getIgnoreInvalidExchanges() != null) {
-            resequencer.setIgnoreInvalidExchanges(config.getIgnoreInvalidExchanges());
-        }
-        return resequencer;
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/RethrowDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RethrowDefinition.java
index e96db8a..96a32e1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RethrowDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RethrowDefinition.java
@@ -51,14 +51,4 @@ public class RethrowDefinition extends NoOutputDefinition<RethrowDefinition> {
         return "rethrow[]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return exchange -> {
-            Exception e = exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Exception.class);
-            if (e != null) {
-                throw e;
-            }
-        };
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/RollbackDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RollbackDefinition.java
index 1b8231a..3a6853d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RollbackDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RollbackDefinition.java
@@ -21,10 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.RollbackProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Forces a rollback by stopping routing the message
@@ -66,22 +63,6 @@ public class RollbackDefinition extends NoOutputDefinition<RollbackDefinition> {
         return "rollback";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        boolean isMarkRollbackOnly = getMarkRollbackOnly() != null && getMarkRollbackOnly();
-        boolean isMarkRollbackOnlyLast = getMarkRollbackOnlyLast() != null && getMarkRollbackOnlyLast();
-
-        // validate that only either mark rollbacks is chosen and not both
-        if (isMarkRollbackOnly && isMarkRollbackOnlyLast) {
-            throw new IllegalArgumentException("Only either one of markRollbackOnly and markRollbackOnlyLast is possible to select as true");
-        }
-
-        RollbackProcessor answer = new RollbackProcessor(message);
-        answer.setMarkRollbackOnly(isMarkRollbackOnly);
-        answer.setMarkRollbackOnlyLast(isMarkRollbackOnlyLast);
-        return answer;
-    }
-
     public String getMessage() {
         return message;
     }
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 23ab610..21caa60 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
@@ -17,9 +17,7 @@
 package org.apache.camel.model;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
-import java.util.StringTokenizer;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -34,30 +32,18 @@ import javax.xml.bind.annotation.XmlType;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
 import org.apache.camel.ErrorHandlerFactory;
-import org.apache.camel.FailedToCreateRouteException;
-import org.apache.camel.NoSuchEndpointException;
-import org.apache.camel.Route;
 import org.apache.camel.ServiceStatus;
 import org.apache.camel.ShutdownRoute;
 import org.apache.camel.ShutdownRunningTask;
-import org.apache.camel.builder.AdviceWithRouteBuilder;
-import org.apache.camel.builder.AdviceWithTask;
 import org.apache.camel.builder.ErrorHandlerBuilderRef;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.impl.DefaultRouteContext;
 import org.apache.camel.model.rest.RestBindingDefinition;
 import org.apache.camel.model.rest.RestDefinition;
-import org.apache.camel.processor.interceptor.HandleFault;
+import org.apache.camel.reifier.RouteReifier;
 import org.apache.camel.spi.AsEndpointUri;
-import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.RoutePolicy;
-import org.apache.camel.spi.RoutePolicyFactory;
-import org.apache.camel.spi.Transformer;
-import org.apache.camel.spi.Validator;
 import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * A Camel route
@@ -165,6 +151,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * Returns the status of the route if it has been registered with a {@link CamelContext}
      */
+    @Deprecated
     public ServiceStatus getStatus(CamelContext camelContext) {
         if (camelContext != null) {
             ServiceStatus answer = camelContext.getRouteController().getRouteStatus(this.getId());
@@ -176,6 +163,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
         return null;
     }
 
+    @Deprecated
     public boolean isStartable(CamelContext camelContext) {
         ServiceStatus status = getStatus(camelContext);
         if (status == null) {
@@ -185,6 +173,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
         }
     }
 
+    @Deprecated
     public boolean isStoppable(CamelContext camelContext) {
         ServiceStatus status = getStatus(camelContext);
         if (status == null) {
@@ -194,35 +183,6 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
         }
     }
 
-    public List<RouteContext> addRoutes(ModelCamelContext camelContext, Collection<Route> routes) throws Exception {
-        List<RouteContext> answer = new ArrayList<>();
-
-        ErrorHandlerFactory handler = camelContext.getErrorHandlerFactory();
-        if (handler != null) {
-            setErrorHandlerBuilderIfNull(handler);
-        }
-
-        for (FromDefinition fromType : inputs) {
-            RouteContext routeContext;
-            try {
-                routeContext = addRoutes(camelContext, routes, fromType);
-            } catch (FailedToCreateRouteException e) {
-                throw e;
-            } catch (Exception e) {
-                // wrap in exception which provide more details about which route was failing
-                throw new FailedToCreateRouteException(getId(), RouteDefinitionHelper.getRouteMessage(toString()), e);
-            }
-            answer.add(routeContext);
-        }
-        return answer;
-    }
-
-
-    public Endpoint resolveEndpoint(CamelContext camelContext, String uri) throws NoSuchEndpointException {
-        ObjectHelper.notNull(camelContext, "CamelContext");
-        return CamelContextHelper.getMandatoryEndpoint(camelContext, uri);
-    }
-
     public RouteDefinition adviceWith(CamelContext camelContext, RouteBuilder builder) throws Exception {
         return adviceWith(camelContext.adapt(ModelCamelContext.class), builder);
     }
@@ -249,64 +209,10 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
      * @param builder      the route builder
      * @return a new route which is this route merged with the route builder
      * @throws Exception can be thrown from the route builder
-     * @see AdviceWithRouteBuilder
+     * @see org.apache.camel.builder.AdviceWithRouteBuilder
      */
     public RouteDefinition adviceWith(ModelCamelContext camelContext, RouteBuilder builder) throws Exception {
-        ObjectHelper.notNull(camelContext, "CamelContext");
-        ObjectHelper.notNull(builder, "RouteBuilder");
-
-        log.debug("AdviceWith route before: {}", this);
-
-        // inject this route into the advice route builder so it can access this route
-        // and offer features to manipulate the route directly
-        if (builder instanceof AdviceWithRouteBuilder) {
-            ((AdviceWithRouteBuilder) builder).setOriginalRoute(this);
-        }
-
-        // configure and prepare the routes from the builder
-        RoutesDefinition routes = builder.configureRoutes(camelContext);
-
-        log.debug("AdviceWith routes: {}", routes);
-
-        // we can only advice with a route builder without any routes
-        if (!builder.getRouteCollection().getRoutes().isEmpty()) {
-            throw new IllegalArgumentException("You can only advice from a RouteBuilder which has no existing routes."
-                    + " Remove all routes from the route builder.");
-        }
-        // we can not advice with error handlers (if you added a new error handler in the route builder)
-        // we must check the error handler on builder is not the same as on camel context, as that would be the default
-        // context scoped error handler, in case no error handlers was configured
-        if (builder.getRouteCollection().getErrorHandlerFactory() != null
-                && camelContext.getErrorHandlerFactory() != builder.getRouteCollection().getErrorHandlerFactory()) {
-            throw new IllegalArgumentException("You can not advice with error handlers. Remove the error handlers from the route builder.");
-        }
-
-        String beforeAsXml = ModelHelper.dumpModelAsXml(camelContext, this);
-
-        // stop and remove this existing route
-        camelContext.removeRouteDefinition(this);
-
-        // any advice with tasks we should execute first?
-        if (builder instanceof AdviceWithRouteBuilder) {
-            List<AdviceWithTask> tasks = ((AdviceWithRouteBuilder) builder).getAdviceWithTasks();
-            for (AdviceWithTask task : tasks) {
-                task.task();
-            }
-        }
-
-        // now merge which also ensures that interceptors and the likes get mixed in correctly as well
-        RouteDefinition merged = routes.route(this);
-
-        // add the new merged route
-        camelContext.addRouteDefinition(merged);
-
-        // log the merged route at info level to make it easier to end users to spot any mistakes they may have made
-        log.info("AdviceWith route after: {}", merged);
-
-        String afterAsXml = ModelHelper.dumpModelAsXml(camelContext, merged);
-        log.info("Adviced route before/after as XML:\n{}\n{}", beforeAsXml, afterAsXml);
-
-        return merged;
+        return new RouteReifier(this).adviceWith(camelContext, builder);
     }
 
     // Fluent API
@@ -672,7 +578,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
 
     /**
      * 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.
+     * at runtime, camel look for a required {@link org.apache.camel.spi.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}.
      *
@@ -691,7 +597,8 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists, and then applies
+     * {@link org.apache.camel.spi.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}.
      *
@@ -711,7 +618,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists.
      *
      * @see org.apache.camel.spi.Transformer
      *
@@ -728,7 +635,8 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists, and then applies
+     * {@link org.apache.camel.spi.Validator} as well.
      *
      * @see org.apache.camel.spi.Transformer
      * @see org.apache.camel.spi.Validator
@@ -745,7 +653,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
 
     /**
      * 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.
+     * at runtime, camel look for a required {@link org.apache.camel.spi.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}.
      *
@@ -764,7 +672,8 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists, and then applies
+     * {@link org.apache.camel.spi.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}.
      * 
@@ -784,7 +693,7 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists.
      *
      * @see org.apache.camel.spi.Transformer
      *
@@ -801,7 +710,8 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
     /**
      * 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.
+     * {@link org.apache.camel.spi.Transformer} and apply if exists, and then applies
+     * {@link org.apache.camel.spi.Validator} as well.
      * 
      * @see org.apache.camel.spi.Transformer
      * @see org.apache.camel.spi.Validator
@@ -987,6 +897,10 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
         return autoStartup;
     }
 
+    /**
+     * TODO: move this somewhere else
+     */
+    @Deprecated
     public boolean isAutoStartup(CamelContext camelContext) throws Exception {
         if (getAutoStartup() == null) {
             // should auto startup by default
@@ -1192,159 +1106,6 @@ public class RouteDefinition extends ProcessorDefinition<RouteDefinition> {
         this.routeProperties = routeProperties;
     }
 
-    // Implementation methods
-    // -------------------------------------------------------------------------
-    protected RouteContext addRoutes(CamelContext camelContext, Collection<Route> routes, FromDefinition fromType) throws Exception {
-        RouteContext routeContext = new DefaultRouteContext(camelContext, this, fromType, routes);
-
-        // configure tracing
-        if (trace != null) {
-            Boolean isTrace = CamelContextHelper.parseBoolean(camelContext, getTrace());
-            if (isTrace != null) {
-                routeContext.setTracing(isTrace);
-                if (isTrace) {
-                    log.debug("Tracing is enabled on route: {}", getId());
-                    // tracing is added in the DefaultChannel so we can enable it on the fly
-                }
-            }
-        }
-
-        // configure message history
-        if (messageHistory != null) {
-            Boolean isMessageHistory = CamelContextHelper.parseBoolean(camelContext, getMessageHistory());
-            if (isMessageHistory != null) {
-                routeContext.setMessageHistory(isMessageHistory);
-                if (isMessageHistory) {
-                    log.debug("Message history is enabled on route: {}", getId());
-                }
-            }
-        }
-
-        // configure Log EIP mask
-        if (logMask != null) {
-            Boolean isLogMask = CamelContextHelper.parseBoolean(camelContext, getLogMask());
-            if (isLogMask != null) {
-                routeContext.setLogMask(isLogMask);
-                if (isLogMask) {
-                    log.debug("Security mask for Logging is enabled on route: {}", getId());
-                }
-            }
-        }
-
-        // configure stream caching
-        if (streamCache != null) {
-            Boolean isStreamCache = CamelContextHelper.parseBoolean(camelContext, getStreamCache());
-            if (isStreamCache != null) {
-                routeContext.setStreamCaching(isStreamCache);
-                if (isStreamCache) {
-                    log.debug("StreamCaching is enabled on route: {}", getId());
-                }
-            }
-        }
-
-        // configure handle fault
-        if (handleFault != null) {
-            Boolean isHandleFault = CamelContextHelper.parseBoolean(camelContext, getHandleFault());
-            if (isHandleFault != null) {
-                routeContext.setHandleFault(isHandleFault);
-                if (isHandleFault) {
-                    log.debug("HandleFault is enabled on route: {}", getId());
-                    // only add a new handle fault if not already a global configured on camel context
-                    if (HandleFault.getHandleFault(camelContext) == null) {
-                        addInterceptStrategy(new HandleFault());
-                    }
-                }
-            }
-        }
-
-        // configure delayer
-        if (delayer != null) {
-            Long delayer = CamelContextHelper.parseLong(camelContext, getDelayer());
-            if (delayer != null) {
-                routeContext.setDelayer(delayer);
-                if (delayer > 0) {
-                    log.debug("Delayer is enabled with: {} ms. on route: {}", delayer, getId());
-                } else {
-                    log.debug("Delayer is disabled on route: {}", getId());
-                }
-            }
-        }
-
-        // configure route policy
-        if (routePolicies != null && !routePolicies.isEmpty()) {
-            for (RoutePolicy policy : routePolicies) {
-                log.debug("RoutePolicy is enabled: {} on route: {}", policy, getId());
-                routeContext.getRoutePolicyList().add(policy);
-            }
-        }
-        if (routePolicyRef != null) {
-            StringTokenizer policyTokens = new StringTokenizer(routePolicyRef, ",");
-            while (policyTokens.hasMoreTokens()) {
-                String ref = policyTokens.nextToken().trim();
-                RoutePolicy policy = CamelContextHelper.mandatoryLookup(camelContext, ref, RoutePolicy.class);
-                log.debug("RoutePolicy is enabled: {} on route: {}", policy, getId());
-                routeContext.getRoutePolicyList().add(policy);
-            }
-        }
-        if (camelContext.getRoutePolicyFactories() != null) {
-            for (RoutePolicyFactory factory : camelContext.getRoutePolicyFactories()) {
-                RoutePolicy policy = factory.createRoutePolicy(camelContext, getId(), this);
-                if (policy != null) {
-                    log.debug("RoutePolicy is enabled: {} on route: {}", policy, getId());
-                    routeContext.getRoutePolicyList().add(policy);
-                }
-            }
-        }
-
-        // configure auto startup
-        Boolean isAutoStartup = CamelContextHelper.parseBoolean(camelContext, getAutoStartup());
-        if (isAutoStartup != null) {
-            log.debug("Using AutoStartup {} on route: {}", isAutoStartup, getId());
-            routeContext.setAutoStartup(isAutoStartup);
-        }
-
-        // configure shutdown
-        if (shutdownRoute != null) {
-            log.debug("Using ShutdownRoute {} on route: {}", getShutdownRoute(), getId());
-            routeContext.setShutdownRoute(getShutdownRoute());
-        }
-        if (shutdownRunningTask != null) {
-            log.debug("Using ShutdownRunningTask {} on route: {}", getShutdownRunningTask(), getId());
-            routeContext.setShutdownRunningTask(getShutdownRunningTask());
-        }
-
-        // should inherit the intercept strategies we have defined
-        routeContext.setInterceptStrategies(this.getInterceptStrategies());
-        // force endpoint resolution
-        routeContext.getEndpoint();
-        for (LifecycleStrategy strategy : camelContext.getLifecycleStrategies()) {
-            strategy.onRouteContextCreate(routeContext);
-        }
-
-        // validate route has output processors
-        if (!ProcessorDefinitionHelper.hasOutputs(outputs, true)) {
-            RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-            String at = fromType.toString();
-            Exception cause = new IllegalArgumentException("Route " + route.getId() + " has no output processors."
-                    + " You need to add outputs to the route such as to(\"log:foo\").");
-            throw new FailedToCreateRouteException(route.getId(), RouteDefinitionHelper.getRouteMessage(route.toString()), at, cause);
-        }
-
-        List<ProcessorDefinition<?>> list = new ArrayList<>(outputs);
-        for (ProcessorDefinition<?> output : list) {
-            try {
-                output.addRoutes(routeContext, routes);
-            } catch (Exception e) {
-                RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-                throw new FailedToCreateRouteException(route.getId(), RouteDefinitionHelper.getRouteMessage(route.toString()), output.toString(), e);
-            }
-        }
-
-        routeContext.commit();
-        return routeContext;
-    }
-
-
     // ****************************
     // Static helpers
     // ****************************
diff --git a/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java b/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java
index b8495ce..f99c1fc 100644
--- a/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/RoutingSlipDefinition.java
@@ -24,15 +24,10 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.AsyncProcessor;
-import org.apache.camel.ErrorHandlerFactory;
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.model.language.HeaderExpression;
-import org.apache.camel.processor.RoutingSlip;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Routes a message through a series of steps that are pre-determined (the slip)
@@ -88,30 +83,6 @@ public class RoutingSlipDefinition<Type extends ProcessorDefinition<Type>> exten
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expression = getExpression().createExpression(routeContext);
-        String delimiter = getUriDelimiter() != null ? getUriDelimiter() : DEFAULT_DELIMITER;
-
-        RoutingSlip routingSlip = new RoutingSlip(routeContext.getCamelContext(), expression, delimiter);
-        if (getIgnoreInvalidEndpoints() != null) {
-            routingSlip.setIgnoreInvalidEndpoints(getIgnoreInvalidEndpoints());
-        }
-        if (getCacheSize() != null) {
-            routingSlip.setCacheSize(getCacheSize());
-        }
-
-        // and wrap this in an error handler
-        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
-        ErrorHandlerFactory builder = route.getErrorHandlerBuilder();
-        // create error handler (create error handler directly to keep it light weight,
-        // instead of using ProcessorDefinition.wrapInErrorHandler)
-        AsyncProcessor errorHandler = (AsyncProcessor) builder.createErrorHandler(routeContext, routingSlip.newRoutingSlipProcessorForErrorHandler());
-        routingSlip.setErrorHandler(errorHandler);
-
-        return routingSlip;
-    }
-
-    @Override
     public List<ProcessorDefinition<?>> getOutputs() {
         return Collections.emptyList();
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/SagaDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SagaDefinition.java
index 93917aa..cb21e36 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SagaDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SagaDefinition.java
@@ -18,9 +18,6 @@ package org.apache.camel.model;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -30,17 +27,9 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
-import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.processor.saga.SagaProcessorBuilder;
 import org.apache.camel.saga.CamelSagaService;
-import org.apache.camel.saga.CamelSagaStep;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
 import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -82,54 +71,6 @@ public class SagaDefinition extends OutputDefinition<SagaDefinition> {
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Optional<Endpoint> compensationEndpoint = Optional.ofNullable(this.compensation)
-                .map(SagaActionUriDefinition::getUri)
-                .map(routeContext::resolveEndpoint);
-
-        Optional<Endpoint> completionEndpoint = Optional.ofNullable(this.completion)
-                .map(SagaActionUriDefinition::getUri)
-                .map(routeContext::resolveEndpoint);
-
-        Map<String, Expression> optionsMap = new TreeMap<>();
-        if (this.options != null) {
-            for (SagaOptionDefinition optionDef : this.options) {
-                String optionName = optionDef.getOptionName();
-                Expression expr = optionDef.getExpression();
-                optionsMap.put(optionName, expr);
-            }
-        }
-
-        CamelSagaStep step = new CamelSagaStep(compensationEndpoint, completionEndpoint, optionsMap, Optional.ofNullable(timeoutInMilliseconds));
-
-        SagaPropagation propagation = this.propagation;
-        if (propagation == null) {
-            // default propagation mode
-            propagation = SagaPropagation.REQUIRED;
-        }
-
-        SagaCompletionMode completionMode = this.completionMode;
-        if (completionMode == null) {
-            // default completion mode
-            completionMode = SagaCompletionMode.defaultCompletionMode();
-        }
-
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-        CamelSagaService camelSagaService = findSagaService(routeContext.getCamelContext());
-
-        camelSagaService.registerStep(step);
-
-        return new SagaProcessorBuilder()
-                .camelContext(routeContext.getCamelContext())
-                .childProcessor(childProcessor)
-                .sagaService(camelSagaService)
-                .step(step)
-                .propagation(propagation)
-                .completionMode(completionMode)
-                .build();
-    }
-
-    @Override
     public boolean isAbstract() {
         return true;
     }
@@ -306,25 +247,6 @@ public class SagaDefinition extends OutputDefinition<SagaDefinition> {
 
     // Utils
 
-    protected CamelSagaService findSagaService(CamelContext context) {
-        CamelSagaService sagaService = getSagaService();
-        if (sagaService != null) {
-            return sagaService;
-        }
-
-        sagaService = context.hasService(CamelSagaService.class);
-        if (sagaService != null) {
-            return sagaService;
-        }
-
-        sagaService = CamelContextHelper.findByType(context, CamelSagaService.class);
-        if (sagaService != null) {
-            return sagaService;
-        }
-
-        throw new RuntimeCamelException("Cannot find a CamelSagaService");
-    }
-
     protected String description() {
         StringBuilder desc = new StringBuilder();
         addField(desc, "compensation", compensation);
diff --git a/camel-core/src/main/java/org/apache/camel/model/SamplingDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SamplingDefinition.java
index 1fc64f7..69a9a4e 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SamplingDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SamplingDefinition.java
@@ -25,11 +25,8 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
-import org.apache.camel.Processor;
 import org.apache.camel.builder.xml.TimeUnitAdapter;
-import org.apache.camel.processor.SamplingThrottler;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Extract a sample of the messages passing through a route
@@ -82,21 +79,6 @@ public class SamplingDefinition extends OutputDefinition<SamplingDefinition> {
         return "sample[" + description() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-        
-        if (messageFrequency != null) {
-            return new SamplingThrottler(childProcessor, messageFrequency);
-        } else {
-            // should default be 1 sample period
-            long time = getSamplePeriod() != null ? getSamplePeriod() : 1L;
-            // should default be in seconds
-            TimeUnit tu = getUnits() != null ? getUnits() : TimeUnit.SECONDS;
-            return new SamplingThrottler(childProcessor, time, tu);
-        }
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/ScriptDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ScriptDefinition.java
index 1dcb0f8..536c7fa 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ScriptDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ScriptDefinition.java
@@ -21,11 +21,8 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.ScriptProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Executes a script from a language which does not change the message body.
@@ -57,12 +54,6 @@ public class ScriptDefinition extends NoOutputExpressionNode {
         return "script[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expr = getExpression().createExpression(routeContext);
-        return new ScriptProcessor(expr);
-    }
-
     /**
      * Expression to return the transformed message body (the new message body to use)
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java
index fdb4db5..487db7f 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SendDefinition.java
@@ -23,11 +23,7 @@ import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Endpoint;
 import org.apache.camel.ExchangePattern;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.SendProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Sends the message to an endpoint
@@ -47,20 +43,6 @@ public abstract class SendDefinition<Type extends ProcessorDefinition<Type>> ext
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Endpoint endpoint = resolveEndpoint(routeContext);
-        return new SendProcessor(endpoint, getPattern());
-    }
-
-    public Endpoint resolveEndpoint(RouteContext context) {
-        if (endpoint == null) {
-            return context.resolveEndpoint(getUri());
-        } else {
-            return endpoint;
-        }
-    }
-
-    @Override
     public String getEndpointUri() {
         if (uri != null) {
             return uri;
diff --git a/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java
index 83b561c..688fc23 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SetBodyDefinition.java
@@ -21,11 +21,8 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.SetBodyProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Sets the contents of the message body
@@ -57,12 +54,6 @@ public class SetBodyDefinition extends NoOutputExpressionNode {
         return "setBody[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expr = getExpression().createExpression(routeContext);
-        return new SetBodyProcessor(expr);
-    }
-
     /**
      * Expression that returns the new body to use
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SetExchangePatternDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SetExchangePatternDefinition.java
index 69357d5..dd6e3da 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SetExchangePatternDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SetExchangePatternDefinition.java
@@ -20,13 +20,9 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.ExchangePattern;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.ExchangePatternProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Sets the exchange pattern on the message exchange
@@ -37,9 +33,7 @@ import org.apache.camel.spi.RouteContext;
 public class SetExchangePatternDefinition extends NoOutputDefinition<SetExchangePatternDefinition> {
     @XmlAttribute(required = true)
     private ExchangePattern pattern;
-    @XmlTransient
-    private ExchangePatternProcessor processor;
-    
+
     public SetExchangePatternDefinition() {
     }
 
@@ -78,12 +72,4 @@ public class SetExchangePatternDefinition extends NoOutputDefinition<SetExchange
         return "setExchangePattern[" + pattern + "]";
     }
    
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        if (processor == null) {
-            processor = new ExchangePatternProcessor(getPattern());
-        }
-        return processor;
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/SetFaultBodyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SetFaultBodyDefinition.java
index 7005bad..01a19b5 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SetFaultBodyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SetFaultBodyDefinition.java
@@ -21,11 +21,8 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
-import org.apache.camel.builder.ProcessorBuilder;
 import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  *  Sets the contents of a fault message's body
@@ -57,12 +54,6 @@ public class SetFaultBodyDefinition extends NoOutputExpressionNode {
         return "setFaultBody[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expr = getExpression().createExpression(routeContext);
-        return ProcessorBuilder.setFaultBody(expr);
-    }
-
     /**
      * Expression that returns the new fault body to use
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java
index 401135d..0de90c6 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SetHeaderDefinition.java
@@ -22,13 +22,9 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionBuilder;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.SetHeaderProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Sets the value of a message header
@@ -73,14 +69,6 @@ public class SetHeaderDefinition extends NoOutputExpressionNode {
         return "setHeader[" + getHeaderName() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(headerName, "headerName");
-        Expression expr = getExpression().createExpression(routeContext);
-        Expression nameExpr = ExpressionBuilder.parseSimpleOrFallbackToConstantExpression(getHeaderName(), routeContext.getCamelContext());
-        return new SetHeaderProcessor(nameExpr, expr);
-    }
-
     /**
      * Expression to return the value of the header
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java
index 73430b0..b8cbf0a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SetPropertyDefinition.java
@@ -22,13 +22,9 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionBuilder;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.SetPropertyProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Sets a named property on the message exchange
@@ -73,14 +69,6 @@ public class SetPropertyDefinition extends NoOutputExpressionNode {
         return "setProperty[" + getPropertyName() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        ObjectHelper.notNull(getPropertyName(), "propertyName", this);
-        Expression expr = getExpression().createExpression(routeContext);
-        Expression nameExpr = ExpressionBuilder.parseSimpleOrFallbackToConstantExpression(getPropertyName(), routeContext.getCamelContext());
-        return new SetPropertyProcessor(nameExpr, expr);
-    }
-
     /**
      * Expression to return the value of the message exchange property
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SortDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SortDefinition.java
index ae4739e..e175c50 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SortDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SortDefinition.java
@@ -25,14 +25,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.SortProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.ObjectHelper;
-
-import static org.apache.camel.builder.ExpressionBuilder.bodyExpression;
 
 /**
  * Sorts the contents of the message
@@ -73,29 +67,6 @@ public class SortDefinition<T> extends NoOutputExpressionNode {
         return "sort[" + getExpression() + "]";
     }
 
-    @Override
-    @SuppressWarnings("unchecked")
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // lookup in registry
-        if (org.apache.camel.util.ObjectHelper.isNotEmpty(comparatorRef)) {
-            comparator = routeContext.getCamelContext().getRegistry().lookupByNameAndType(comparatorRef, Comparator.class);
-        }
-
-        // if no comparator then default on to string representation
-        if (comparator == null) {
-            comparator = (Comparator<T>) ObjectHelper::compare;
-        }
-
-        // if no expression provided then default to body expression
-        Expression exp;
-        if (getExpression() == null) {
-            exp = bodyExpression();
-        } else {
-            exp = getExpression().createExpression(routeContext);
-        }
-        return new SortProcessor<T>(exp, getComparator());
-    }
-
     /**
      * Optional expression to sort by something else than the message body
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/SplitDefinition.java b/camel-core/src/main/java/org/apache/camel/model/SplitDefinition.java
index 12c490d..9f6d25d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/SplitDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/SplitDefinition.java
@@ -24,17 +24,11 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContextAware;
 import org.apache.camel.Expression;
 import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.Splitter;
 import org.apache.camel.processor.aggregate.AggregationStrategy;
-import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
-import org.apache.camel.processor.aggregate.ShareUnitOfWorkAggregationStrategy;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
 
 /**
  * Splits a single message into many sub-messages.
@@ -100,65 +94,6 @@ public class SplitDefinition extends ExpressionNode implements ExecutorServiceAw
         return "split[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-        aggregationStrategy = createAggregationStrategy(routeContext);
-
-        boolean isParallelProcessing = getParallelProcessing() != null && getParallelProcessing();
-        boolean isStreaming = getStreaming() != null && getStreaming();
-        boolean isShareUnitOfWork = getShareUnitOfWork() != null && getShareUnitOfWork();
-        boolean isParallelAggregate = getParallelAggregate() != null && getParallelAggregate();
-        boolean isStopOnAggregateException = getStopOnAggregateException() != null && getStopOnAggregateException();
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, isParallelProcessing);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "Split", this, isParallelProcessing);
-
-        long timeout = getTimeout() != null ? getTimeout() : 0;
-        if (timeout > 0 && !isParallelProcessing) {
-            throw new IllegalArgumentException("Timeout is used but ParallelProcessing has not been enabled.");
-        }
-        if (onPrepareRef != null) {
-            onPrepare = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onPrepareRef, Processor.class);
-        }
-
-        Expression exp = getExpression().createExpression(routeContext);
-
-        Splitter answer = new Splitter(routeContext.getCamelContext(), exp, childProcessor, aggregationStrategy,
-                            isParallelProcessing, threadPool, shutdownThreadPool, isStreaming, isStopOnException(),
-                            timeout, onPrepare, isShareUnitOfWork, isParallelAggregate, isStopOnAggregateException);
-        return answer;
-    }
-
-    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
-        AggregationStrategy strategy = getAggregationStrategy();
-        if (strategy == null && strategyRef != null) {
-            Object aggStrategy = routeContext.lookup(strategyRef, Object.class);
-            if (aggStrategy instanceof AggregationStrategy) {
-                strategy = (AggregationStrategy) aggStrategy;
-            } else if (aggStrategy != null) {
-                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, getStrategyMethodName());
-                if (getStrategyMethodAllowNull() != null) {
-                    adapter.setAllowNullNewExchange(getStrategyMethodAllowNull());
-                    adapter.setAllowNullOldExchange(getStrategyMethodAllowNull());
-                }
-                strategy = adapter;
-            } else {
-                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + strategyRef);
-            }
-        }
-
-        if (strategy instanceof CamelContextAware) {
-            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
-        }
-
-        if (strategy != null && shareUnitOfWork != null && shareUnitOfWork) {
-            // wrap strategy in share unit of work
-            strategy = new ShareUnitOfWorkAggregationStrategy(strategy);
-        }
-
-        return strategy;
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
diff --git a/camel-core/src/main/java/org/apache/camel/model/StopDefinition.java b/camel-core/src/main/java/org/apache/camel/model/StopDefinition.java
index 8b58b16..9c47cc5 100644
--- a/camel-core/src/main/java/org/apache/camel/model/StopDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/StopDefinition.java
@@ -20,10 +20,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
-import org.apache.camel.processor.StopProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Stops the processing of the current message
@@ -37,11 +34,6 @@ public class StopDefinition extends NoOutputDefinition<StopDefinition> {
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return new StopProcessor();
-    }
-
-    @Override
     public String getShortName() {
         return "stop";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ThreadsDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ThreadsDefinition.java
index a6114ce..5d7eb62 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ThreadsDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ThreadsDefinition.java
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.model;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
@@ -28,16 +26,9 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
-import org.apache.camel.Processor;
-import org.apache.camel.util.concurrent.ThreadPoolRejectedPolicy;
-import org.apache.camel.builder.ThreadPoolProfileBuilder;
 import org.apache.camel.builder.xml.TimeUnitAdapter;
-import org.apache.camel.processor.Pipeline;
-import org.apache.camel.processor.ThreadsProcessor;
-import org.apache.camel.spi.ExecutorServiceManager;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.spi.ThreadPoolProfile;
+import org.apache.camel.util.concurrent.ThreadPoolRejectedPolicy;
 
 /**
  * Specifies that all steps after this node are processed asynchronously
@@ -78,92 +69,6 @@ public class ThreadsDefinition extends OutputDefinition<ThreadsDefinition> imple
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // the threads name
-        String name = getThreadName() != null ? getThreadName() : "Threads";
-        // prefer any explicit configured executor service
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, true);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, name, this, false);
-
-        // resolve what rejected policy to use
-        ThreadPoolRejectedPolicy policy = resolveRejectedPolicy(routeContext);
-        if (policy == null) {
-            if (callerRunsWhenRejected == null || callerRunsWhenRejected) {
-                // should use caller runs by default if not configured
-                policy = ThreadPoolRejectedPolicy.CallerRuns;
-            } else {
-                policy = ThreadPoolRejectedPolicy.Abort;
-            }
-        }
-        log.debug("Using ThreadPoolRejectedPolicy: {}", policy);
-
-        // if no explicit then create from the options
-        if (threadPool == null) {
-            ExecutorServiceManager manager = routeContext.getCamelContext().getExecutorServiceManager();
-            // create the thread pool using a builder
-            ThreadPoolProfile profile = new ThreadPoolProfileBuilder(name)
-                    .poolSize(getPoolSize())
-                    .maxPoolSize(getMaxPoolSize())
-                    .keepAliveTime(getKeepAliveTime(), getTimeUnit())
-                    .maxQueueSize(getMaxQueueSize())
-                    .rejectedPolicy(policy)
-                    .allowCoreThreadTimeOut(getAllowCoreThreadTimeOut())
-                    .build();
-            threadPool = manager.newThreadPool(this, name, profile);
-            shutdownThreadPool = true;
-        } else {
-            if (getThreadName() != null && !getThreadName().equals("Threads")) {
-                throw new IllegalArgumentException("ThreadName and executorServiceRef options cannot be used together.");
-            }
-            if (getPoolSize() != null) {
-                throw new IllegalArgumentException("PoolSize and executorServiceRef options cannot be used together.");
-            }
-            if (getMaxPoolSize() != null) {
-                throw new IllegalArgumentException("MaxPoolSize and executorServiceRef options cannot be used together.");
-            }
-            if (getKeepAliveTime() != null) {
-                throw new IllegalArgumentException("KeepAliveTime and executorServiceRef options cannot be used together.");
-            }
-            if (getTimeUnit() != null) {
-                throw new IllegalArgumentException("TimeUnit and executorServiceRef options cannot be used together.");
-            }
-            if (getMaxQueueSize() != null) {
-                throw new IllegalArgumentException("MaxQueueSize and executorServiceRef options cannot be used together.");
-            }
-            if (getRejectedPolicy() != null) {
-                throw new IllegalArgumentException("RejectedPolicy and executorServiceRef options cannot be used together.");
-            }
-            if (getAllowCoreThreadTimeOut() != null) {
-                throw new IllegalArgumentException("AllowCoreThreadTimeOut and executorServiceRef options cannot be used together.");
-            }
-        }
-
-        ThreadsProcessor thread = new ThreadsProcessor(routeContext.getCamelContext(), threadPool, shutdownThreadPool, policy);
-
-        List<Processor> pipe = new ArrayList<>(2);
-        pipe.add(thread);
-        pipe.add(createChildProcessor(routeContext, true));
-        // wrap in nested pipeline so this appears as one processor
-        // (recipient list definition does this as well)
-        return new Pipeline(routeContext.getCamelContext(), pipe) {
-            @Override
-            public String toString() {
-                return "Threads[" + getOutputs() + "]";
-            }
-        };
-    }
-
-    protected ThreadPoolRejectedPolicy resolveRejectedPolicy(RouteContext routeContext) {
-        if (getExecutorServiceRef() != null && getRejectedPolicy() == null) {
-            ThreadPoolProfile threadPoolProfile = routeContext.getCamelContext().getExecutorServiceManager().getThreadPoolProfile(getExecutorServiceRef());
-            if (threadPoolProfile != null) {
-                return threadPoolProfile.getRejectedPolicy();
-            }
-        }
-        return getRejectedPolicy();
-    }
-
-    @Override
     public String getShortName() {
         return "threads";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ThrottleDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ThrottleDefinition.java
index 7d32bfc..068e9fa 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ThrottleDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ThrottleDefinition.java
@@ -17,7 +17,6 @@
 package org.apache.camel.model;
 
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ScheduledExecutorService;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -28,12 +27,9 @@ import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionBuilder;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.Throttler;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Controls the rate at which messages are passed to the next node in the route
@@ -98,50 +94,6 @@ public class ThrottleDefinition extends ExpressionNode implements ExecutorServic
         return "throttle[" + description() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        boolean async = getAsyncDelayed() != null && getAsyncDelayed();
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, true);
-        ScheduledExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredScheduledExecutorService(routeContext, "Throttle", this, true);
-        
-        // should be default 1000 millis
-        long period = getTimePeriodMillis() != null ? getTimePeriodMillis() : 1000L;
-
-        // max requests per period is mandatory
-        Expression maxRequestsExpression = createMaxRequestsPerPeriodExpression(routeContext);
-        if (maxRequestsExpression == null) {
-            throw new IllegalArgumentException("MaxRequestsPerPeriod expression must be provided on " + this);
-        }
-        
-        Expression correlation = null;
-        if (correlationExpression != null) {
-            correlation = correlationExpression.createExpression(routeContext);
-        }
-
-        boolean reject = getRejectExecution() != null && getRejectExecution();
-        Throttler answer = new Throttler(routeContext.getCamelContext(), childProcessor, maxRequestsExpression, period, threadPool, shutdownThreadPool, reject, correlation);
-
-        answer.setAsyncDelayed(async);
-        if (getCallerRunsWhenRejected() == null) {
-            // should be true by default
-            answer.setCallerRunsWhenRejected(true);
-        } else {
-            answer.setCallerRunsWhenRejected(getCallerRunsWhenRejected());
-        }
-
-        return answer;
-    }
-
-    private Expression createMaxRequestsPerPeriodExpression(RouteContext routeContext) {
-        ExpressionDefinition expr = getExpression();
-        if (expr != null) {
-            return expr.createExpression(routeContext);
-        }
-        return null;
-    }
-    
     // Fluent API
     // -------------------------------------------------------------------------
     /**
diff --git a/camel-core/src/main/java/org/apache/camel/model/ThrowExceptionDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ThrowExceptionDefinition.java
index 415677d..e39cd84 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ThrowExceptionDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ThrowExceptionDefinition.java
@@ -22,11 +22,7 @@ import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.Processor;
-import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.processor.ThrowExceptionProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Throws an exception
@@ -68,26 +64,6 @@ public class ThrowExceptionDefinition extends NoOutputDefinition<ThrowExceptionD
         return "throwException[" + description() + "]";
     }
     
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        if (ref != null && exception == null) {
-            this.exception = routeContext.getCamelContext().getRegistry().lookupByNameAndType(ref, Exception.class);
-        }
-
-        if (exceptionType != null && exceptionClass == null) {
-            try {
-                this.exceptionClass = routeContext.getCamelContext().getClassResolver().resolveMandatoryClass(exceptionType, Exception.class);
-            } catch (ClassNotFoundException e) {
-                throw RuntimeCamelException.wrapRuntimeCamelException(e);
-            }
-        }
-
-        if (exception == null && exceptionClass == null) {
-            throw new IllegalArgumentException("exception or exceptionClass/exceptionType must be configured on: " + this);
-        }
-        return new ThrowExceptionProcessor(exception, exceptionClass, message);
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ToDynamicDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ToDynamicDefinition.java
index 11aea0b..d635f3c 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ToDynamicDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ToDynamicDefinition.java
@@ -16,26 +16,13 @@
  */
 package org.apache.camel.model;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.ExchangePattern;
-import org.apache.camel.Expression;
-import org.apache.camel.NoSuchLanguageException;
-import org.apache.camel.Processor;
-import org.apache.camel.builder.ExpressionBuilder;
-import org.apache.camel.processor.SendDynamicProcessor;
-import org.apache.camel.spi.Language;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.StringHelper;
 
 /**
  * Sends the message to a dynamic endpoint
@@ -51,8 +38,6 @@ import org.apache.camel.util.StringHelper;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ToDynamicDefinition extends NoOutputDefinition<ToDynamicDefinition> {
 
-    private static final Pattern RAW_PATTERN = Pattern.compile("RAW\\([^\\)]+\\)");
-
     @XmlAttribute @Metadata(required = "true")
     private String uri;
     @XmlAttribute
@@ -72,64 +57,6 @@ public class ToDynamicDefinition extends NoOutputDefinition<ToDynamicDefinition>
     }
 
     @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        StringHelper.notEmpty(uri, "uri", this);
-
-        Expression exp = createExpression(routeContext);
-
-        SendDynamicProcessor processor = new SendDynamicProcessor(uri, exp);
-        processor.setCamelContext(routeContext.getCamelContext());
-        processor.setPattern(pattern);
-        if (cacheSize != null) {
-            processor.setCacheSize(cacheSize);
-        }
-        if (ignoreInvalidEndpoint != null) {
-            processor.setIgnoreInvalidEndpoint(ignoreInvalidEndpoint);
-        }
-        return processor;
-    }
-
-    protected Expression createExpression(RouteContext routeContext) {
-        List<Expression> list = new ArrayList<>();
-
-        String[] parts = safeSplitRaw(uri);
-        for (String part : parts) {
-            // the part may have optional language to use, so you can mix languages
-            String value = StringHelper.after(part, "language:");
-            if (value != null) {
-                String before = StringHelper.before(value, ":");
-                String after = StringHelper.after(value, ":");
-                if (before != null && after != null) {
-                    // maybe its a language, must have language: as prefix
-                    try {
-                        Language partLanguage = routeContext.getCamelContext().resolveLanguage(before);
-                        if (partLanguage != null) {
-                            Expression exp = partLanguage.createExpression(after);
-                            list.add(exp);
-                            continue;
-                        }
-                    } catch (NoSuchLanguageException e) {
-                        // ignore
-                    }
-                }
-            }
-            // fallback and use simple language
-            Language lan = routeContext.getCamelContext().resolveLanguage("simple");
-            Expression exp = lan.createExpression(part);
-            list.add(exp);
-        }
-
-        Expression exp;
-        if (list.size() == 1) {
-            exp = list.get(0);
-        } else {
-            exp = ExpressionBuilder.concatExpression(list);
-        }
-
-        return exp;
-    }
-
-    @Override
     public String getShortName() {
         return "toD";
     }
@@ -227,76 +154,4 @@ public class ToDynamicDefinition extends NoOutputDefinition<ToDynamicDefinition>
         this.allowOptimisedComponents = allowOptimisedComponents;
     }
 
-    // Utilities
-    // -------------------------------------------------------------------------
-
-    private static class Pair {
-        int left;
-        int right;
-        Pair(int left, int right) {
-            this.left = left;
-            this.right = right;
-        }
-    }
-
-    private static List<Pair> checkRAW(String s) {
-        Matcher matcher = RAW_PATTERN.matcher(s);
-        List<Pair> answer = new ArrayList<>();
-        // Check all occurrences
-        while (matcher.find()) {
-            answer.add(new Pair(matcher.start(), matcher.end() - 1));
-        }
-        return answer;
-    }
-
-    private static boolean isRaw(int index, List<Pair>pairs) {
-        for (Pair pair : pairs) {
-            if (index < pair.left) {
-                return false;
-            } else {
-                if (index >= pair.left) {
-                    if (index <= pair.right) {
-                        return true;
-                    } else {
-                        continue;
-                    }
-                }
-            }
-        }
-        return false;
-    }
-
-    /**
-     * We need to split the string safely for each + sign, but avoid splitting within RAW(...).
-     */
-    private static String[] safeSplitRaw(String s) {
-        List<String> list = new ArrayList<>();
-
-        if (!s.contains("+")) {
-            // no plus sign so there is only one part, so no need to split
-            list.add(s);
-        } else {
-            // there is a plus sign so we need to split in a safe manner
-            List<Pair> rawPairs = checkRAW(s);
-            StringBuilder sb = new StringBuilder();
-            char chars[] = s.toCharArray();
-            for (int i = 0; i < chars.length; i++) {
-                char ch = chars[i];
-                if (ch != '+' || isRaw(i, rawPairs)) {
-                    sb.append(ch);
-                } else {
-                    list.add(sb.toString());
-                    sb.setLength(0);
-                }
-            }
-            // any leftover?
-            if (sb.length() > 0) {
-                list.add(sb.toString());
-                sb.setLength(0);
-            }
-        }
-
-        return list.toArray(new String[list.size()]);
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/TransactedDefinition.java b/camel-core/src/main/java/org/apache/camel/model/TransactedDefinition.java
index 6bbb3b6..00e465c 100644
--- a/camel-core/src/main/java/org/apache/camel/model/TransactedDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/TransactedDefinition.java
@@ -16,28 +16,16 @@
  */
 package org.apache.camel.model;
 
-import java.lang.reflect.Method;
-import java.util.Map;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.NoSuchBeanException;
-import org.apache.camel.Processor;
-import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.Service;
-import org.apache.camel.processor.WrapProcessor;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Policy;
-import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.TransactedPolicy;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.support.ObjectHelper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 
 /**
  * Enables transaction on the route
@@ -55,8 +43,6 @@ public class TransactedDefinition extends OutputDefinition<TransactedDefinition>
     @XmlTransient
     public static final String PROPAGATION_REQUIRED = "PROPAGATION_REQUIRED";
 
-    private static final Logger LOG = LoggerFactory.getLogger(TransactedDefinition.class);
-
     @XmlTransient
     protected Class<? extends Policy> type = TransactedPolicy.class;
     @XmlAttribute
@@ -122,6 +108,10 @@ public class TransactedDefinition extends OutputDefinition<TransactedDefinition>
         return true;
     }
 
+    public Policy getPolicy() {
+        return policy;
+    }
+
     public String getRef() {
         return ref;
     }
@@ -130,6 +120,10 @@ public class TransactedDefinition extends OutputDefinition<TransactedDefinition>
         this.ref = ref;
     }
 
+    public Class<? extends Policy> getType() {
+        return type;
+    }
+
     /**
      * Sets a policy type that this definition should scope within.
      * <p/>
@@ -157,107 +151,4 @@ public class TransactedDefinition extends OutputDefinition<TransactedDefinition>
         return this;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Policy policy = resolvePolicy(routeContext);
-        org.apache.camel.util.ObjectHelper.notNull(policy, "policy", this);
-
-        // before wrap
-        policy.beforeWrap(routeContext, this);
-
-        // create processor after the before wrap
-        Processor childProcessor = this.createChildProcessor(routeContext, true);
-
-        // wrap
-        Processor target = policy.wrap(routeContext, childProcessor);
-
-        if (!(target instanceof Service)) {
-            // wrap the target so it becomes a service and we can manage its lifecycle
-            target = new WrapProcessor(target, childProcessor);
-        }
-        return target;
-    }
-
-    protected Policy resolvePolicy(RouteContext routeContext) {
-        if (policy != null) {
-            return policy;
-        }
-        return doResolvePolicy(routeContext, getRef(), type);
-    }
-
-    protected static Policy doResolvePolicy(RouteContext routeContext, String ref, Class<? extends Policy> type) {
-        // explicit ref given so lookup by it
-        if (org.apache.camel.util.ObjectHelper.isNotEmpty(ref)) {
-            return CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), ref, Policy.class);
-        }
-
-        // no explicit reference given from user so we can use some convention over configuration here
-
-        // try to lookup by scoped type
-        Policy answer = null;
-        if (type != null) {
-            // try find by type, note that this method is not supported by all registry
-            Map<String, ?> types = routeContext.lookupByType(type);
-            if (types.size() == 1) {
-                // only one policy defined so use it
-                Object found = types.values().iterator().next();
-                if (type.isInstance(found)) {
-                    return type.cast(found);
-                }
-            }
-        }
-
-        // for transacted routing try the default REQUIRED name
-        if (type == TransactedPolicy.class) {
-            // still not found try with the default name PROPAGATION_REQUIRED
-            answer = routeContext.lookup(PROPAGATION_REQUIRED, TransactedPolicy.class);
-        }
-
-        // this logic only applies if we are a transacted policy
-        // still no policy found then try lookup the platform transaction manager and use it as policy
-        if (answer == null && type == TransactedPolicy.class) {
-            Class<?> tmClazz = routeContext.getCamelContext().getClassResolver().resolveClass("org.springframework.transaction.PlatformTransactionManager");
-            if (tmClazz != null) {
-                // see if we can find the platform transaction manager in the registry
-                Map<String, ?> maps = routeContext.lookupByType(tmClazz);
-                if (maps.size() == 1) {
-                    // only one platform manager then use it as default and create a transacted
-                    // policy with it and default to required
-
-                    // as we do not want dependency on spring jars in the camel-core we use
-                    // reflection to lookup classes and create new objects and call methods
-                    // as this is only done during route building it does not matter that we
-                    // use reflection as performance is no a concern during route building
-                    Object transactionManager = maps.values().iterator().next();
-                    LOG.debug("One instance of PlatformTransactionManager found in registry: {}", transactionManager);
-                    Class<?> txClazz = routeContext.getCamelContext().getClassResolver().resolveClass("org.apache.camel.spring.spi.SpringTransactionPolicy");
-                    if (txClazz != null) {
-                        LOG.debug("Creating a new temporary SpringTransactionPolicy using the PlatformTransactionManager: {}", transactionManager);
-                        TransactedPolicy txPolicy = ObjectHelper.newInstance(txClazz, TransactedPolicy.class);
-                        Method method;
-                        try {
-                            method = txClazz.getMethod("setTransactionManager", tmClazz);
-                        } catch (NoSuchMethodException e) {
-                            throw new RuntimeCamelException("Cannot get method setTransactionManager(PlatformTransactionManager) on class: " + txClazz);
-                        }
-                        ObjectHelper.invokeMethod(method, txPolicy, transactionManager);
-                        return txPolicy;
-                    } else {
-                        // camel-spring is missing on the classpath
-                        throw new RuntimeCamelException("Cannot create a transacted policy as camel-spring.jar is not on the classpath!");
-                    }
-                } else {
-                    if (maps.isEmpty()) {
-                        throw new NoSuchBeanException(null, "PlatformTransactionManager");
-                    } else {
-                        throw new IllegalArgumentException("Found " + maps.size() + " PlatformTransactionManager in registry. "
-                                + "Cannot determine which one to use. Please configure a TransactionTemplate on the transacted policy.");
-                    }
-                }
-            }
-        }
-
-        return answer;
-    }
-
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java b/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java
index e5b79e2..a708ea1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/TransformDefinition.java
@@ -21,11 +21,8 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.TransformProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Transforms the message body based on an expression
@@ -57,12 +54,6 @@ public class TransformDefinition extends NoOutputExpressionNode {
         return "transform[" + getExpression() + "]";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Expression expr = getExpression().createExpression(routeContext);
-        return new TransformProcessor(expr);
-    }
-
     /**
      * Expression to return the transformed message body (the new message body to use)
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/TryDefinition.java b/camel-core/src/main/java/org/apache/camel/model/TryDefinition.java
index 9e01d84..9a85bc1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/TryDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/TryDefinition.java
@@ -27,11 +27,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.Predicate;
-import org.apache.camel.Processor;
-import org.apache.camel.processor.TryProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Marks the beginning of a try, catch, finally block
@@ -67,35 +64,6 @@ public class TryDefinition extends OutputDefinition<TryDefinition> {
         return "doTry";
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        Processor tryProcessor = createOutputsProcessor(routeContext, getOutputsWithoutCatches());
-        if (tryProcessor == null) {
-            throw new IllegalArgumentException("Definition has no children on " + this);
-        }
-
-        List<Processor> catchProcessors = new ArrayList<>();
-        if (catchClauses != null) {
-            for (CatchDefinition catchClause : catchClauses) {
-                catchProcessors.add(createProcessor(routeContext, catchClause));
-            }
-        }
-
-        FinallyDefinition finallyDefinition = finallyClause;
-        if (finallyDefinition == null) {
-            finallyDefinition = new FinallyDefinition();
-            finallyDefinition.setParent(this);
-        }
-        Processor finallyProcessor = createProcessor(routeContext, finallyDefinition);
-
-        // must have either a catch or finally
-        if (finallyClause == null && catchClauses == null) {
-            throw new IllegalArgumentException("doTry must have one or more catch or finally blocks on " + this);
-        }
-
-        return new TryProcessor(tryProcessor, catchProcessors, finallyProcessor);
-    }
-
     // Fluent API
     // -------------------------------------------------------------------------
 
@@ -208,7 +176,7 @@ public class TryDefinition extends OutputDefinition<TryDefinition> {
     }
 
     @Override
-    protected void preCreateProcessor() {
+    public void preCreateProcessor() {
         // force re-creating initialization to ensure its up-to-date
         initialized = false;
         checkInitialized();
diff --git a/camel-core/src/main/java/org/apache/camel/model/UnmarshalDefinition.java b/camel-core/src/main/java/org/apache/camel/model/UnmarshalDefinition.java
index f4195fc..18d1987 100644
--- a/camel-core/src/main/java/org/apache/camel/model/UnmarshalDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/UnmarshalDefinition.java
@@ -22,7 +22,6 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Processor;
 import org.apache.camel.model.dataformat.ASN1DataFormat;
 import org.apache.camel.model.dataformat.AvroDataFormat;
 import org.apache.camel.model.dataformat.Base64DataFormat;
@@ -61,10 +60,7 @@ import org.apache.camel.model.dataformat.XmlRpcDataFormat;
 import org.apache.camel.model.dataformat.YAMLDataFormat;
 import org.apache.camel.model.dataformat.ZipDataFormat;
 import org.apache.camel.model.dataformat.ZipFileDataFormat;
-import org.apache.camel.processor.UnmarshalProcessor;
-import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Converts the message data received from the wire into a format that Apache Camel processors can consume
@@ -160,9 +156,4 @@ public class UnmarshalDefinition extends NoOutputDefinition<UnmarshalDefinition>
         this.dataFormatType = dataFormatType;
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) {
-        DataFormat dataFormat = DataFormatDefinition.getDataFormat(routeContext, getDataFormatType(), null);
-        return new UnmarshalProcessor(dataFormat);
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java
index 960ccc0..d7df9a7 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ValidateDefinition.java
@@ -23,10 +23,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.camel.Expression;
 import org.apache.camel.Predicate;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.validation.PredicateValidatingProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Validates a message based on an expression
@@ -62,12 +60,6 @@ public class ValidateDefinition extends NoOutputExpressionNode {
         return "validate[" + getExpression() + "]";
     }
 
-    @Override
-    public PredicateValidatingProcessor createProcessor(RouteContext routeContext) throws Exception {
-        Predicate pred = getExpression().createPredicate(routeContext);
-        return new PredicateValidatingProcessor(pred);
-    }
-
     /**
      * Expression to use for validation as a predicate. The expression should return either <tt>true</tt> or <tt>false</tt>.
      * If returning <tt>false</tt> the message is invalid and an exception is thrown.
diff --git a/camel-core/src/main/java/org/apache/camel/model/WhenDefinition.java b/camel-core/src/main/java/org/apache/camel/model/WhenDefinition.java
index 34b2f58..7940366 100644
--- a/camel-core/src/main/java/org/apache/camel/model/WhenDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/WhenDefinition.java
@@ -20,10 +20,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Predicate;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.FilterProcessor;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Triggers a route when an expression evaluates to <tt>true</tt>
@@ -73,11 +71,6 @@ public class WhenDefinition extends ExpressionNode {
         return "when[" + description() + "]";
     }
 
-    @Override
-    public FilterProcessor createProcessor(RouteContext routeContext) throws Exception {
-        return createFilterProcessor(routeContext);
-    }
-
     /**
      * Expression used as the predicate to evaluate whether this when should trigger and route the message or not.
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/WhenSkipSendToEndpointDefinition.java b/camel-core/src/main/java/org/apache/camel/model/WhenSkipSendToEndpointDefinition.java
index 77b88fd..46f2ee5 100644
--- a/camel-core/src/main/java/org/apache/camel/model/WhenSkipSendToEndpointDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/WhenSkipSendToEndpointDefinition.java
@@ -18,12 +18,9 @@ package org.apache.camel.model;
 
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.Exchange;
-import org.apache.camel.Predicate;
 import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Predicate to determine if the message should be sent or not to the endpoint, when using interceptSentToEndpoint.
@@ -36,26 +33,6 @@ public class WhenSkipSendToEndpointDefinition extends WhenDefinition {
         return "whenSkipSendToEndpoint";
     }
 
-    @Override
-    protected Predicate createPredicate(RouteContext routeContext) {
-        // we need to keep track whether the when matches or not, so delegate
-        // the predicate and add the matches result as a property on the exchange
-        final Predicate delegate = super.createPredicate(routeContext);
-        return new Predicate() {
-            @Override
-            public boolean matches(Exchange exchange) {
-                boolean matches = delegate.matches(exchange);
-                exchange.setProperty(Exchange.INTERCEPT_SEND_TO_ENDPOINT_WHEN_MATCHED, matches);
-                return matches;
-            }
-
-            @Override
-            public String toString() {
-                return delegate.toString();
-            }
-        };
-    }
-
     /**
      * Expression used as the predicate to evaluate whether the message should be sent or not to the endpoint
      */
diff --git a/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java b/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java
index d152cf4..9798b44 100644
--- a/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/WireTapDefinition.java
@@ -31,13 +31,7 @@ import javax.xml.bind.annotation.XmlTransient;
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Expression;
 import org.apache.camel.Processor;
-import org.apache.camel.builder.ExpressionBuilder;
-import org.apache.camel.processor.CamelInternalProcessor;
-import org.apache.camel.processor.SendDynamicProcessor;
-import org.apache.camel.processor.WireTapProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
 
 /**
  * Routes a copy of a message (or creates a new message) to a secondary destination while continue routing the original message.
@@ -70,66 +64,7 @@ public class WireTapDefinition<Type extends ProcessorDefinition<Type>> extends T
     public WireTapDefinition() {
     }
 
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        // executor service is mandatory for wire tap
-        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, this, true);
-        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "WireTap", this, true);
-
-        // must use InOnly for WireTap
-        setPattern(ExchangePattern.InOnly);
-
-        // create the send dynamic producer to send to the wire tapped endpoint
-        SendDynamicProcessor dynamicTo = (SendDynamicProcessor) super.createProcessor(routeContext);
-
-        // create error handler we need to use for processing the wire tapped
-        Processor target = wrapInErrorHandler(routeContext, dynamicTo);
-
-        // and wrap in unit of work
-        CamelInternalProcessor internal = new CamelInternalProcessor(target);
-        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
-
-        // is true bt default
-        boolean isCopy = getCopy() == null || getCopy();
-
-        WireTapProcessor answer = new WireTapProcessor(dynamicTo, internal, getPattern(), threadPool, shutdownThreadPool, isDynamic());
-        answer.setCopy(isCopy);
-        if (newExchangeProcessorRef != null) {
-            newExchangeProcessor = routeContext.mandatoryLookup(newExchangeProcessorRef, Processor.class);
-        }
-        if (newExchangeProcessor != null) {
-            answer.addNewExchangeProcessor(newExchangeProcessor);
-        }
-        if (newExchangeExpression != null) {
-            answer.setNewExchangeExpression(newExchangeExpression.createExpression(routeContext));
-        }
-        if (headers != null && !headers.isEmpty()) {
-            for (SetHeaderDefinition header : headers) {
-                Processor processor = createProcessor(routeContext, header);
-                answer.addNewExchangeProcessor(processor);
-            }
-        }
-        if (onPrepareRef != null) {
-            onPrepare = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), onPrepareRef, Processor.class);
-        }
-        if (onPrepare != null) {
-            answer.setOnPrepare(onPrepare);
-        }
-
-        return answer;
-    }
-
-    @Override
-    protected Expression createExpression(RouteContext routeContext) {
-        // whether to use dynamic or static uri
-        if (isDynamic()) {
-            return super.createExpression(routeContext);
-        } else {
-            return ExpressionBuilder.constantExpression(getUri());
-        }
-    }
-
-    private boolean isDynamic() {
+    public boolean isDynamic() {
         // its dynamic by default
         return dynamicUri == null || dynamicUri;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinition.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinition.java
index 53a40c8..30fb272 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinition.java
@@ -16,10 +16,6 @@
  */
 package org.apache.camel.model.cloud;
 
-import java.util.Optional;
-import java.util.function.Function;
-import java.util.function.Supplier;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -28,38 +24,15 @@ import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.CamelContextAware;
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Expression;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.ExpressionClause;
 import org.apache.camel.cloud.ServiceChooser;
-import org.apache.camel.cloud.ServiceChooserAware;
 import org.apache.camel.cloud.ServiceDiscovery;
-import org.apache.camel.cloud.ServiceDiscoveryAware;
-import org.apache.camel.cloud.ServiceExpressionFactory;
 import org.apache.camel.cloud.ServiceFilter;
-import org.apache.camel.cloud.ServiceFilterAware;
 import org.apache.camel.cloud.ServiceLoadBalancer;
-import org.apache.camel.impl.cloud.DefaultServiceCallExpression;
-import org.apache.camel.impl.cloud.DefaultServiceCallProcessor;
-import org.apache.camel.impl.cloud.DefaultServiceLoadBalancer;
-import org.apache.camel.impl.cloud.HealthyServiceFilter;
-import org.apache.camel.impl.cloud.PassThroughServiceFilter;
-import org.apache.camel.impl.cloud.RandomServiceChooser;
-import org.apache.camel.impl.cloud.RoundRobinServiceChooser;
-import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.NoOutputDefinition;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.util.ObjectHelper;
-import org.apache.camel.util.function.Suppliers;
-import org.apache.camel.util.function.ThrowingHelper;
-
-import static org.apache.camel.support.CamelContextHelper.findByType;
-import static org.apache.camel.support.CamelContextHelper.lookup;
 
 /**
  * To call remote services
@@ -754,403 +727,4 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
         return this;
     }
 
-    // *****************************
-    // Processor Factory
-    // *****************************
-
-    @Override
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        final CamelContext camelContext = routeContext.getCamelContext();
-        final ServiceDiscovery serviceDiscovery = retrieveServiceDiscovery(camelContext);
-        final ServiceFilter serviceFilter = retrieveServiceFilter(camelContext);
-        final ServiceChooser serviceChooser = retrieveServiceChooser(camelContext);
-        final ServiceLoadBalancer loadBalancer = retrieveLoadBalancer(camelContext);
-
-        CamelContextAware.trySetCamelContext(serviceDiscovery, camelContext);
-        CamelContextAware.trySetCamelContext(serviceFilter, camelContext);
-        CamelContextAware.trySetCamelContext(serviceChooser, camelContext);
-        CamelContextAware.trySetCamelContext(loadBalancer, camelContext);
-
-        if (loadBalancer instanceof ServiceDiscoveryAware) {
-            ((ServiceDiscoveryAware) loadBalancer).setServiceDiscovery(serviceDiscovery);
-        }
-        if (loadBalancer instanceof ServiceFilterAware) {
-            ((ServiceFilterAware) loadBalancer).setServiceFilter(serviceFilter);
-        }
-        if (loadBalancer instanceof ServiceChooserAware) {
-            ((ServiceChooserAware) loadBalancer).setServiceChooser(serviceChooser);
-        }
-
-        // The component is used to configure the default scheme to use (eg camel component name).
-        // The component configured on EIP takes precedence vs configured on configuration.
-        String endpointScheme = this.component;
-        if (endpointScheme == null) {
-            ServiceCallConfigurationDefinition conf = retrieveConfig(camelContext);
-            if (conf != null) {
-                endpointScheme = conf.getComponent();
-            }
-        }
-        if (endpointScheme == null) {
-            ServiceCallConfigurationDefinition conf = retrieveDefaultConfig(camelContext);
-            if (conf != null) {
-                endpointScheme = conf.getComponent();
-            }
-        }
-
-        // The uri is used to tweak the uri.
-        // The uri configured on EIP takes precedence vs configured on configuration.
-        String endpointUri = this.uri;
-        if (endpointUri == null) {
-            ServiceCallConfigurationDefinition conf = retrieveConfig(camelContext);
-            if (conf != null) {
-                endpointUri = conf.getUri();
-            }
-        }
-        if (endpointUri == null) {
-            ServiceCallConfigurationDefinition conf = retrieveDefaultConfig(camelContext);
-            if (conf != null) {
-                endpointUri = conf.getUri();
-            }
-        }
-
-        // Service name is mandatory
-        ObjectHelper.notNull(name, "Service name");
-
-        endpointScheme = ThrowingHelper.applyIfNotEmpty(endpointScheme, camelContext::resolvePropertyPlaceholders, () -> ServiceCallDefinitionConstants.DEFAULT_COMPONENT);
-        endpointUri = ThrowingHelper.applyIfNotEmpty(endpointUri, camelContext::resolvePropertyPlaceholders, () -> null);
-
-        return new DefaultServiceCallProcessor(
-            camelContext,
-            camelContext.resolvePropertyPlaceholders(name),
-            endpointScheme,
-            endpointUri,
-            pattern,
-            loadBalancer,
-            retrieveExpression(camelContext, endpointScheme));
-    }
-
-    // *****************************
-    // Helpers
-    // *****************************
-
-    private ServiceCallConfigurationDefinition retrieveDefaultConfig(CamelContext camelContext) {
-        // check if a default configuration is bound to the registry
-        ServiceCallConfigurationDefinition config = camelContext.adapt(ModelCamelContext.class).getServiceCallConfiguration(null);
-
-        if (config == null) {
-            // Or if it is in the registry
-            config = lookup(
-                camelContext,
-                ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_CONFIG_ID,
-                ServiceCallConfigurationDefinition.class);
-        }
-
-        if (config == null) {
-            // If no default is set either by searching by name or bound to the
-            // camel context, assume that if there is a single instance in the
-            // registry, that is the default one
-            config = findByType(camelContext, ServiceCallConfigurationDefinition.class);
-        }
-
-        return config;
-    }
-
-    private ServiceCallConfigurationDefinition retrieveConfig(CamelContext camelContext) {
-        ServiceCallConfigurationDefinition config = null;
-        if (configurationRef != null) {
-            // lookup in registry firstNotNull
-            config = lookup(camelContext, configurationRef, ServiceCallConfigurationDefinition.class);
-            if (config == null) {
-                // and fallback as service configuration
-                config = camelContext.adapt(ModelCamelContext.class).getServiceCallConfiguration(configurationRef);
-            }
-        }
-
-        return config;
-    }
-
-    // ******************************************
-    // ServiceDiscovery
-    // ******************************************
-
-    private ServiceDiscovery retrieveServiceDiscovery(CamelContext camelContext, Function<CamelContext, ServiceCallConfigurationDefinition> function) throws Exception {
-        ServiceDiscovery answer = null;
-
-        ServiceCallConfigurationDefinition config = function.apply(camelContext);
-        if (config != null) {
-            if (config.getServiceDiscoveryConfiguration() != null) {
-                answer = config.getServiceDiscoveryConfiguration().newInstance(camelContext);
-            } else {
-                answer = retrieve(
-                    ServiceDiscovery.class,
-                    camelContext,
-                    config::getServiceDiscovery,
-                    config::getServiceDiscoveryRef
-                );
-            }
-        }
-
-        return answer;
-    }
-
-    private ServiceDiscovery retrieveServiceDiscovery(CamelContext camelContext) throws Exception {
-        return Suppliers.firstNotNull(
-            () -> (serviceDiscoveryConfiguration != null) ? serviceDiscoveryConfiguration.newInstance(camelContext) : null,
-            // Local configuration
-            () -> retrieve(ServiceDiscovery.class, camelContext, this::getServiceDiscovery, this::getServiceDiscoveryRef),
-            // Linked configuration
-            () -> retrieveServiceDiscovery(camelContext, this::retrieveConfig),
-            // Default configuration
-            () -> retrieveServiceDiscovery(camelContext, this::retrieveDefaultConfig),
-            // Check if there is a single instance in the registry
-            () -> findByType(camelContext, ServiceDiscovery.class),
-            // From registry
-            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_DISCOVERY_ID, ServiceDiscovery.class)
-        ).orElseGet(
-            // Default, that's s little ugly but a load balancer may live without
-            // (i.e. the Ribbon one) so let's delegate the null check to the actual
-            // impl.
-            () -> null
-        );
-    }
-
-    // ******************************************
-    // ServiceFilter
-    // ******************************************
-
-    private ServiceFilter retrieveServiceFilter(CamelContext camelContext, Function<CamelContext, ServiceCallConfigurationDefinition> function) throws Exception {
-        ServiceFilter answer = null;
-
-        ServiceCallConfigurationDefinition config = function.apply(camelContext);
-        if (config != null) {
-            if (config.getServiceFilterConfiguration() != null) {
-                answer = config.getServiceFilterConfiguration().newInstance(camelContext);
-            } else {
-                answer = retrieve(
-                    ServiceFilter.class,
-                    camelContext,
-                    config::getServiceFilter,
-                    config::getServiceFilterRef
-                );
-            }
-
-            if (answer == null) {
-                String ref = config.getServiceFilterRef();
-                if (ObjectHelper.equal("healthy", ref, true)) {
-                    answer = new HealthyServiceFilter();
-                } else if (ObjectHelper.equal("pass-through", ref, true)) {
-                    answer = new PassThroughServiceFilter();
-                } else if (ObjectHelper.equal("passthrough", ref, true)) {
-                    answer = new PassThroughServiceFilter();
-                }
-            }
-        }
-
-        return answer;
-    }
-
-    private ServiceFilter retrieveServiceFilter(CamelContext camelContext) throws Exception {
-        return Suppliers.firstNotNull(
-            () -> (serviceFilterConfiguration != null) ? serviceFilterConfiguration.newInstance(camelContext) : null,
-            // Local configuration
-            () -> retrieve(ServiceFilter.class, camelContext, this::getServiceFilter, this::getServiceFilterRef),
-            // Linked configuration
-            () -> retrieveServiceFilter(camelContext, this::retrieveConfig),
-            // Default configuration
-            () -> retrieveServiceFilter(camelContext, this::retrieveDefaultConfig),
-            // Check if there is a single instance in the registry
-            () -> findByType(camelContext, ServiceFilter.class),
-            // From registry
-            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_FILTER_ID, ServiceFilter.class)
-        ).orElseGet(
-            // Default
-            () -> new HealthyServiceFilter()
-        );
-    }
-
-    // ******************************************
-    // ServiceChooser
-    // ******************************************
-
-    private ServiceChooser retrieveServiceChooser(CamelContext camelContext, Function<CamelContext, ServiceCallConfigurationDefinition> function) throws Exception {
-        ServiceChooser answer = null;
-
-        ServiceCallConfigurationDefinition config = function.apply(camelContext);
-        if (config != null) {
-            answer = retrieve(
-                ServiceChooser.class,
-                camelContext,
-                config::getServiceChooser,
-                config::getServiceChooserRef
-            );
-
-            if (answer == null) {
-                String ref = config.getServiceChooserRef();
-                if (ObjectHelper.equal("roundrobin", ref, true)) {
-                    answer = new RoundRobinServiceChooser();
-                } else if (ObjectHelper.equal("round-robin", ref, true)) {
-                    answer = new RoundRobinServiceChooser();
-                } else if (ObjectHelper.equal("random", ref, true)) {
-                    answer = new RandomServiceChooser();
-                }
-            }
-        }
-
-        return answer;
-    }
-
-    private ServiceChooser retrieveServiceChooser(CamelContext camelContext) throws Exception {
-        return Suppliers.firstNotNull(
-            // Local configuration
-            () -> retrieve(ServiceChooser.class, camelContext, this::getServiceChooser, this::getServiceChooserRef),
-            // Linked configuration
-            () -> retrieveServiceChooser(camelContext, this::retrieveConfig),
-            // Default configuration
-            () -> retrieveServiceChooser(camelContext, this::retrieveDefaultConfig),
-            // Check if there is a single instance in the registry
-            () -> findByType(camelContext, ServiceChooser.class),
-            // From registry
-            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_CHOOSER_ID, ServiceChooser.class)
-        ).orElseGet(
-            // Default
-            () -> new RoundRobinServiceChooser()
-        );
-    }
-
-    // ******************************************
-    // LoadBalancer
-    // ******************************************
-
-    private ServiceLoadBalancer retrieveLoadBalancer(CamelContext camelContext, Function<CamelContext, ServiceCallConfigurationDefinition> function) throws Exception {
-        ServiceLoadBalancer answer = null;
-
-        ServiceCallConfigurationDefinition config = function.apply(camelContext);
-        if (config != null) {
-            if (config.getLoadBalancerConfiguration() != null) {
-                answer = config.getLoadBalancerConfiguration().newInstance(camelContext);
-            } else {
-                answer = retrieve(
-                    ServiceLoadBalancer.class,
-                    camelContext,
-                    config::getLoadBalancer,
-                    config::getLoadBalancerRef
-                );
-            }
-        }
-
-        return answer;
-    }
-
-    private ServiceLoadBalancer retrieveLoadBalancer(CamelContext camelContext) throws Exception {
-        return Suppliers.firstNotNull(
-            () -> (loadBalancerConfiguration != null) ? loadBalancerConfiguration.newInstance(camelContext) : null,
-            // Local configuration
-            () -> retrieve(ServiceLoadBalancer.class, camelContext, this::getLoadBalancer, this::getLoadBalancerRef),
-            // Linked configuration
-            () -> retrieveLoadBalancer(camelContext, this::retrieveConfig),
-            // Default configuration
-            () -> retrieveLoadBalancer(camelContext, this::retrieveDefaultConfig),
-            // Check if there is a single instance in the registry
-            () -> findByType(camelContext, ServiceLoadBalancer.class),
-            // From registry
-            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_LOAD_BALANCER_ID, ServiceLoadBalancer.class)
-        ).orElseGet(
-            // Default
-            () -> new DefaultServiceLoadBalancer()
-        );
-    }
-
-    // ******************************************
-    // Expression
-    // ******************************************
-
-    private Expression retrieveExpression(CamelContext camelContext, Function<CamelContext, ServiceCallConfigurationDefinition> function) throws Exception {
-        Expression answer = null;
-
-        ServiceCallConfigurationDefinition config = function.apply(camelContext);
-        if (config != null) {
-            if (config.getExpressionConfiguration() != null) {
-                answer = config.getExpressionConfiguration().newInstance(camelContext);
-            } else {
-                answer = retrieve(
-                    Expression.class,
-                    camelContext,
-                    config::getExpression,
-                    config::getExpressionRef
-                );
-            }
-        }
-
-        return answer;
-    }
-
-    private Expression retrieveExpression(CamelContext camelContext, String component) throws Exception {
-        Optional<Expression> expression = Suppliers.firstNotNull(
-            () -> (expressionConfiguration != null) ? expressionConfiguration.newInstance(camelContext) : null,
-            // Local configuration
-            () -> retrieve(Expression.class, camelContext, this::getExpression, this::getExpressionRef),
-            // Linked configuration
-            () -> retrieveExpression(camelContext, this::retrieveConfig),
-            // Default configuration
-            () -> retrieveExpression(camelContext, this::retrieveDefaultConfig),
-            // From registry
-            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_EXPRESSION_ID, Expression.class)
-        );
-
-        if (expression.isPresent()) {
-            return expression.get();
-        } else {
-            String lookupName = component + "-service-expression";
-            // First try to find the factory from the registry.
-            ServiceExpressionFactory factory = CamelContextHelper.lookup(camelContext, lookupName, ServiceExpressionFactory.class);
-            if (factory != null) {
-                // If a factory is found in the registry do not re-configure it as
-                // it should be pre-configured.
-                return factory.newInstance(camelContext);
-            } else {
-
-                Class<?> type = null;
-
-                try {
-                    // Then use Service factory.
-                    type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(lookupName);
-                } catch (Exception e) {
-                }
-
-                if (ObjectHelper.isNotEmpty(type)) {
-                    if (ServiceExpressionFactory.class.isAssignableFrom(type)) {
-                        factory = (ServiceExpressionFactory) camelContext.getInjector().newInstance(type);
-                    } else {
-                        throw new IllegalArgumentException(
-                            "Resolving Expression: " + lookupName + " detected type conflict: Not a ServiceExpressionFactory implementation. Found: " + type.getName());
-                    }
-                } else {
-                    // If no factory is found, returns the default
-                    factory = context -> new DefaultServiceCallExpression();
-                }
-
-                return factory.newInstance(camelContext);
-            }
-        }
-    }
-
-    // ************************************
-    // Helpers
-    // ************************************
-
-    private <T> T retrieve(Class<T> type, CamelContext camelContext, Supplier<T> instanceSupplier, Supplier<String> refSupplier) {
-        T answer = null;
-        if (instanceSupplier != null) {
-            answer = instanceSupplier.get();
-        }
-
-        if (answer == null && refSupplier != null) {
-            String ref = refSupplier.get();
-            if (ref != null) {
-                answer = lookup(camelContext, ref, type);
-            }
-        }
-
-        return answer;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CustomLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CustomLoadBalancerDefinition.java
index 9fca2f2..5990be8 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CustomLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/CustomLoadBalancerDefinition.java
@@ -25,9 +25,6 @@ import javax.xml.bind.annotation.XmlTransient;
 import org.apache.camel.model.LoadBalancerDefinition;
 import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.util.StringHelper;
 
 /**
  * Custom load balancer
@@ -56,31 +53,21 @@ public class CustomLoadBalancerDefinition extends LoadBalancerDefinition {
         this.ref = ref;
     }
 
-    public LoadBalancer getLoadBalancer() {
+    public LoadBalancer getCustomLoadBalancer() {
         return loadBalancer;
     }
 
     /**
      * The custom load balancer to use.
      */
-    public void setLoadBalancer(LoadBalancer loadBalancer) {
+    public void setCustomLoadBalancer(LoadBalancer loadBalancer) {
         this.loadBalancer = loadBalancer;
     }
 
     @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        if (loadBalancer != null) {
-            return loadBalancer;
-        }
-
-        StringHelper.notEmpty(ref, "ref", this);
-        return CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), ref, LoadBalancer.class);
-    }
-
-    @Override
     public String toString() {
-        if (loadBalancer != null) {
-            return "CustomLoadBalancer[" + loadBalancer + "]";
+        if (getCustomLoadBalancer() != null) {
+            return "CustomLoadBalancer[" + getCustomLoadBalancer() + "]";
         } else {
             return "CustomLoadBalancer[" + ref + "]";
         }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java
index af6cd5c..b9df87a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/FailoverLoadBalancerDefinition.java
@@ -27,11 +27,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.camel.model.LoadBalancerDefinition;
-import org.apache.camel.processor.loadbalancer.FailOverLoadBalancer;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.util.ObjectHelper;
 
 /**
  * Failover load balancer
@@ -59,44 +55,6 @@ public class FailoverLoadBalancerDefinition extends LoadBalancerDefinition {
     public FailoverLoadBalancerDefinition() {
     }
 
-    @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        FailOverLoadBalancer answer;
-
-        List<Class<?>> classes = new ArrayList<>();
-        if (!exceptionTypes.isEmpty()) {
-            classes.addAll(exceptionTypes);
-        } else if (!exceptions.isEmpty()) {
-            for (String name : exceptions) {
-                Class<?> type = routeContext.getCamelContext().getClassResolver().resolveClass(name);
-                if (type == null) {
-                    throw new IllegalArgumentException("Cannot find class: " + name + " in the classpath");
-                }
-                if (!ObjectHelper.isAssignableFrom(Throwable.class, type)) {
-                    throw new IllegalArgumentException("Class is not an instance of Throwable: " + type);
-                }
-                classes.add(type);
-            }
-        }
-        if (classes.isEmpty()) {
-            answer = new FailOverLoadBalancer();
-        } else {
-            answer = new FailOverLoadBalancer(classes);
-        }
-
-        if (getMaximumFailoverAttempts() != null) {
-            answer.setMaximumFailoverAttempts(getMaximumFailoverAttempts());
-        }
-        if (roundRobin != null) {
-            answer.setRoundRobin(roundRobin);
-        }
-        if (sticky != null) {
-            answer.setSticky(sticky);
-        }
-
-        return answer;
-    }
-
     public List<String> getExceptions() {
         return exceptions;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalancerDefinition.java
index 236101f..a225d33 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalancerDefinition.java
@@ -21,9 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.model.LoadBalancerDefinition;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Random load balancer
@@ -39,11 +37,6 @@ public class RandomLoadBalancerDefinition extends LoadBalancerDefinition {
     }
 
     @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        return new org.apache.camel.processor.loadbalancer.RandomLoadBalancer();
-    }
-
-    @Override
     public String toString() {
         return "RandomLoadBalancer";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalancerDefinition.java
index c6353bd..a7740a3 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalancerDefinition.java
@@ -21,9 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.model.LoadBalancerDefinition;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Round robin load balancer
@@ -40,11 +38,6 @@ public class RoundRobinLoadBalancerDefinition extends LoadBalancerDefinition {
     }
 
     @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        return new org.apache.camel.processor.loadbalancer.RoundRobinLoadBalancer();
-    }
-
-    @Override
     public String toString() {
         return "RoundRobinLoadBalancer";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalancerDefinition.java
index 813060a..9d545e1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalancerDefinition.java
@@ -26,10 +26,7 @@ import org.apache.camel.model.ExpressionNodeHelper;
 import org.apache.camel.model.ExpressionSubElementDefinition;
 import org.apache.camel.model.LoadBalancerDefinition;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
-import org.apache.camel.processor.loadbalancer.StickyLoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Sticky load balancer
@@ -47,11 +44,6 @@ public class StickyLoadBalancerDefinition extends LoadBalancerDefinition {
     public StickyLoadBalancerDefinition() {
     }
 
-    @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        return new StickyLoadBalancer(correlationExpression.createExpression(routeContext));
-    }
-
     public ExpressionSubElementDefinition getCorrelationExpression() {
         return correlationExpression;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalancerDefinition.java
index 6e15723..2ca98cf 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalancerDefinition.java
@@ -21,9 +21,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.model.LoadBalancerDefinition;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Topic load balancer
@@ -39,11 +37,6 @@ public class TopicLoadBalancerDefinition extends LoadBalancerDefinition {
     }
 
     @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        return new org.apache.camel.processor.loadbalancer.TopicLoadBalancer();
-    }
-
-    @Override
     public String toString() {
         return "TopicLoadBalancer";
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/WeightedLoadBalancerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/WeightedLoadBalancerDefinition.java
index cc567ea..0ff9373 100644
--- a/camel-core/src/main/java/org/apache/camel/model/loadbalancer/WeightedLoadBalancerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/loadbalancer/WeightedLoadBalancerDefinition.java
@@ -16,22 +16,13 @@
  */
 package org.apache.camel.model.loadbalancer;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.model.LoadBalancerDefinition;
-import org.apache.camel.processor.loadbalancer.LoadBalancer;
-import org.apache.camel.processor.loadbalancer.WeightedLoadBalancer;
-import org.apache.camel.processor.loadbalancer.WeightedRandomLoadBalancer;
-import org.apache.camel.processor.loadbalancer.WeightedRoundRobinLoadBalancer;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Weighted load balancer
@@ -54,30 +45,6 @@ public class WeightedLoadBalancerDefinition extends LoadBalancerDefinition {
     public WeightedLoadBalancerDefinition() {
     }
 
-    @Override
-    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
-        WeightedLoadBalancer loadBalancer;
-        List<Integer> distributionRatioList = new ArrayList<>();
-        
-        try {
-            String[] ratios = distributionRatio.split(getDistributionRatioDelimiter());
-            for (String ratio : ratios) {
-                distributionRatioList.add(new Integer(ratio.trim()));
-            }
-
-            boolean isRoundRobin = getRoundRobin() != null && getRoundRobin();
-            if (isRoundRobin) {
-                loadBalancer = new WeightedRoundRobinLoadBalancer(distributionRatioList);
-            } else {
-                loadBalancer = new WeightedRandomLoadBalancer(distributionRatioList);
-            }
-        } catch (Exception e) {
-            throw RuntimeCamelException.wrapRuntimeCamelException(e);
-        }
-
-        return loadBalancer;
-    }
-
     public Boolean getRoundRobin() {
         return roundRobin;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/rest/RestBindingDefinition.java b/camel-core/src/main/java/org/apache/camel/model/rest/RestBindingDefinition.java
index e16801d..df48cdd 100644
--- a/camel-core/src/main/java/org/apache/camel/model/rest/RestBindingDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/rest/RestBindingDefinition.java
@@ -21,22 +21,14 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import javax.xml.bind.JAXBContext;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.model.OptionalIdentifiedDefinition;
-import org.apache.camel.processor.RestBindingAdvice;
-import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.RestConfiguration;
-import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.EndpointHelper;
-import org.apache.camel.support.IntrospectionSupport;
 
 /**
  * To configure rest binding
@@ -94,172 +86,6 @@ public class RestBindingDefinition extends OptionalIdentifiedDefinition<RestBind
         return "RestBinding";
     }
 
-    public RestBindingAdvice createRestBindingAdvice(RouteContext routeContext) throws Exception {
-
-        CamelContext context = routeContext.getCamelContext();
-        RestConfiguration config = context.getRestConfiguration(component, true);
-
-        // these options can be overridden per rest verb
-        String mode = config.getBindingMode().name();
-        if (bindingMode != null) {
-            mode = bindingMode.name();
-        }
-        boolean cors = config.isEnableCORS();
-        if (enableCORS != null) {
-            cors = enableCORS;
-        }
-        boolean skip = config.isSkipBindingOnErrorCode();
-        if (skipBindingOnErrorCode != null) {
-            skip = skipBindingOnErrorCode;
-        }
-        boolean validation = config.isClientRequestValidation();
-        if (clientRequestValidation != null) {
-            validation = clientRequestValidation;
-        }
-
-        // cors headers
-        Map<String, String> corsHeaders = config.getCorsHeaders();
-
-        if (mode == null || "off".equals(mode)) {
-            // binding mode is off, so create a off mode binding processor
-            return new RestBindingAdvice(context, null, null, null, null, consumes, produces, mode, skip, validation,
-                cors, corsHeaders, defaultValues, requiredBody != null ? requiredBody : false, requiredQueryParameters, requiredHeaders);
-        }
-
-        // setup json data format
-        DataFormat json = null;
-        DataFormat outJson = null;
-        if (mode.contains("json") || "auto".equals(mode)) {
-            String name = config.getJsonDataFormat();
-            if (name != null) {
-                // must only be a name, not refer to an existing instance
-                Object instance = context.getRegistry().lookupByName(name);
-                if (instance != null) {
-                    throw new IllegalArgumentException("JsonDataFormat name: " + name + " must not be an existing bean instance from the registry");
-                }
-            } else {
-                name = "json-jackson";
-            }
-            // this will create a new instance as the name was not already pre-created
-            json = context.resolveDataFormat(name);
-            outJson = context.resolveDataFormat(name);
-
-            if (json != null) {
-                Class<?> clazz = null;
-                if (type != null) {
-                    String typeName = type.endsWith("[]") ? type.substring(0, type.length() - 2) : type;
-                    clazz = context.getClassResolver().resolveMandatoryClass(typeName);
-                }
-                if (clazz != null) {
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), json, "unmarshalType", clazz);
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), json, "useList", type.endsWith("[]"));
-                }
-                setAdditionalConfiguration(config, context, json, "json.in.");
-
-                Class<?> outClazz = null;
-                if (outType != null) {
-                    String typeName = outType.endsWith("[]") ? outType.substring(0, outType.length() - 2) : outType;
-                    outClazz = context.getClassResolver().resolveMandatoryClass(typeName);
-                }
-                if (outClazz != null) {
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), outJson, "unmarshalType", outClazz);
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), outJson, "useList", outType.endsWith("[]"));
-                }
-                setAdditionalConfiguration(config, context, outJson, "json.out.");
-            }
-        }
-
-        // setup xml data format
-        DataFormat jaxb = null;
-        DataFormat outJaxb = null;
-        if (mode.contains("xml") || "auto".equals(mode)) {
-            String name = config.getXmlDataFormat();
-            if (name != null) {
-                // must only be a name, not refer to an existing instance
-                Object instance = context.getRegistry().lookupByName(name);
-                if (instance != null) {
-                    throw new IllegalArgumentException("XmlDataFormat name: " + name + " must not be an existing bean instance from the registry");
-                }
-            } else {
-                name = "jaxb";
-            }
-            // this will create a new instance as the name was not already pre-created
-            jaxb = context.resolveDataFormat(name);
-            outJaxb = context.resolveDataFormat(name);
-
-            // is xml binding required?
-            if (mode.contains("xml") && jaxb == null) {
-                throw new IllegalArgumentException("XML DataFormat " + name + " not found.");
-            }
-
-            if (jaxb != null) {
-                Class<?> clazz = null;
-                if (type != null) {
-                    String typeName = type.endsWith("[]") ? type.substring(0, type.length() - 2) : type;
-                    clazz = context.getClassResolver().resolveMandatoryClass(typeName);
-                }
-                if (clazz != null) {
-                    JAXBContext jc = JAXBContext.newInstance(clazz);
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), jaxb, "context", jc);
-                }
-                setAdditionalConfiguration(config, context, jaxb, "xml.in.");
-
-                Class<?> outClazz = null;
-                if (outType != null) {
-                    String typeName = outType.endsWith("[]") ? outType.substring(0, outType.length() - 2) : outType;
-                    outClazz = context.getClassResolver().resolveMandatoryClass(typeName);
-                }
-                if (outClazz != null) {
-                    JAXBContext jc = JAXBContext.newInstance(outClazz);
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), outJaxb, "context", jc);
-                } else if (clazz != null) {
-                    // fallback and use the context from the input
-                    JAXBContext jc = JAXBContext.newInstance(clazz);
-                    IntrospectionSupport.setProperty(context.getTypeConverter(), outJaxb, "context", jc);
-                }
-                setAdditionalConfiguration(config, context, outJaxb, "xml.out.");
-            }
-        }
-
-        return new RestBindingAdvice(context, json, jaxb, outJson, outJaxb, consumes, produces, mode, skip, validation,
-            cors, corsHeaders, defaultValues, requiredBody != null ? requiredBody : false, requiredQueryParameters, requiredHeaders);
-    }
-
-    private void setAdditionalConfiguration(RestConfiguration config, CamelContext context,
-                                            DataFormat dataFormat, String prefix) throws Exception {
-        if (config.getDataFormatProperties() != null && !config.getDataFormatProperties().isEmpty()) {
-            // must use a copy as otherwise the options gets removed during introspection setProperties
-            Map<String, Object> copy = new HashMap<>();
-
-            // filter keys on prefix
-            // - either its a known prefix and must match the prefix parameter
-            // - or its a common configuration that we should always use
-            for (Map.Entry<String, Object> entry : config.getDataFormatProperties().entrySet()) {
-                String key = entry.getKey();
-                String copyKey;
-                boolean known = isKeyKnownPrefix(key);
-                if (known) {
-                    // remove the prefix from the key to use
-                    copyKey = key.substring(prefix.length());
-                } else {
-                    // use the key as is
-                    copyKey = key;
-                }
-                if (!known || key.startsWith(prefix)) {
-                    copy.put(copyKey, entry.getValue());
-                }
-            }
-
-            // set reference properties first as they use # syntax that fools the regular properties setter
-            EndpointHelper.setReferenceProperties(context, dataFormat, copy);
-            EndpointHelper.setProperties(context, dataFormat, copy);
-        }
-    }
-
-    private boolean isKeyKnownPrefix(String key) {
-        return key.startsWith("json.in.") || key.startsWith("json.out.") || key.startsWith("xml.in.") || key.startsWith("xml.out.");
-    }
-
     public String getConsumes() {
         return consumes;
     }
@@ -289,6 +115,10 @@ public class RestBindingDefinition extends OptionalIdentifiedDefinition<RestBind
         requiredQueryParameters.add(paramName);
     }
 
+    public Set<String> getRequiredQueryParameters() {
+        return requiredQueryParameters;
+    }
+
     /**
      * Adds a required HTTP header
      *
@@ -301,6 +131,10 @@ public class RestBindingDefinition extends OptionalIdentifiedDefinition<RestBind
         requiredHeaders.add(headerName);
     }
 
+    public Set<String> getRequiredHeaders() {
+        return requiredHeaders;
+    }
+
     public Boolean getRequiredBody() {
         return requiredBody;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/transformer/CustomTransformerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/transformer/CustomTransformerDefinition.java
index 4fb4d8d..431503e 100644
--- a/camel-core/src/main/java/org/apache/camel/model/transformer/CustomTransformerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/transformer/CustomTransformerDefinition.java
@@ -21,7 +21,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Transformer;
 
@@ -42,34 +41,6 @@ public class CustomTransformerDefinition extends TransformerDefinition {
     @XmlAttribute
     private String className;
 
-    @Override
-    protected Transformer doCreateTransformer(CamelContext context) throws Exception {
-        if (ref == null && className == null) {
-            throw new IllegalArgumentException("'ref' or 'className' must be specified for customTransformer");
-        }
-        Transformer transformer;
-        if (ref != null) {
-            transformer = context.getRegistry().lookupByNameAndType(ref, Transformer.class);
-            if (transformer == null) {
-                throw new IllegalArgumentException("Cannot find transformer with ref:" + ref);
-            }
-            if (transformer.getModel() != null || transformer.getFrom() != null || transformer.getTo() != null) {
-                throw new IllegalArgumentException(String.format("Transformer '%s' is already in use. Please check if duplicate transformer exists.", ref));
-            }
-        } else {
-            Class<Transformer> transformerClass = context.getClassResolver().resolveMandatoryClass(className, Transformer.class);
-            if (transformerClass == null) {
-                throw new IllegalArgumentException("Cannot find transformer class: " + className);
-            }
-            transformer = context.getInjector().newInstance(transformerClass);
-
-        }
-        transformer.setCamelContext(context);
-        return transformer.setModel(getScheme())
-                          .setFrom(getFromType())
-                          .setTo(getToType());
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/transformer/DataFormatTransformerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/transformer/DataFormatTransformerDefinition.java
index e4cacd0..e200c25 100644
--- a/camel-core/src/main/java/org/apache/camel/model/transformer/DataFormatTransformerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/transformer/DataFormatTransformerDefinition.java
@@ -23,8 +23,6 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.impl.transformer.DataFormatTransformer;
 import org.apache.camel.model.DataFormatDefinition;
 import org.apache.camel.model.dataformat.ASN1DataFormat;
 import org.apache.camel.model.dataformat.AvroDataFormat;
@@ -67,12 +65,11 @@ import org.apache.camel.model.dataformat.XmlRpcDataFormat;
 import org.apache.camel.model.dataformat.YAMLDataFormat;
 import org.apache.camel.model.dataformat.ZipDataFormat;
 import org.apache.camel.model.dataformat.ZipFileDataFormat;
-import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.Transformer;
 
 /**
- * Represents a {@link DataFormatTransformer} which leverages {@link DataFormat} to perform
+ * Represents a {@link org.apache.camel.impl.transformer.DataFormatTransformer} which leverages
+ * {@link org.apache.camel.spi.DataFormat} to perform
  * transformation. One of the DataFormat 'ref' or DataFormat 'type' needs to be specified.
  * 
  * {@see TransformerDefinition}
@@ -132,16 +129,6 @@ public class DataFormatTransformerDefinition extends TransformerDefinition {
     @XmlAttribute
     private String ref;
 
-    @Override
-    protected Transformer doCreateTransformer(CamelContext context) {
-        return new DataFormatTransformer(context)
-                .setDataFormatType(dataFormatType)
-                .setDataFormatRef(ref)
-                .setModel(getScheme())
-                .setFrom(getFromType())
-                .setTo(getToType());
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/transformer/EndpointTransformerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/transformer/EndpointTransformerDefinition.java
index a5b1acc..5b2e041 100644
--- a/camel-core/src/main/java/org/apache/camel/model/transformer/EndpointTransformerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/transformer/EndpointTransformerDefinition.java
@@ -21,18 +21,12 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
-import org.apache.camel.ExchangePattern;
-import org.apache.camel.impl.transformer.ProcessorTransformer;
-import org.apache.camel.processor.SendProcessor;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.Transformer;
 
 /**
- * Represents an endpoint {@link Transformer} which leverages camel {@link Endpoint} to
- * perform transformation. A {@link ProcessorTransformer} will be created internally
- * with a {@link SendProcessor} which forwards the message to the specified Endpoint.
+ * Represents an endpoint {@link org.apache.camel.spi.Transformer} which leverages camel {@link org.apache.camel.Endpoint} to
+ * perform transformation. A {@link org.apache.camel.impl.transformer.ProcessorTransformer} will be created internally
+ * with a {@link org.apache.camel.processor.SendProcessor} which forwards the message to the specified Endpoint.
  * One of the Endpoint 'ref' or 'uri' needs to be specified.
  * 
  * {@see TransformerDefinition}
@@ -48,18 +42,6 @@ public class EndpointTransformerDefinition extends TransformerDefinition {
     @XmlAttribute
     private String uri;
 
-    @Override
-    protected Transformer doCreateTransformer(CamelContext context) throws Exception {
-        Endpoint endpoint = uri != null ? context.getEndpoint(uri)
-            : context.getRegistry().lookupByNameAndType(ref, Endpoint.class);
-        SendProcessor processor = new SendProcessor(endpoint, ExchangePattern.InOut);
-        return new ProcessorTransformer(context)
-            .setProcessor(processor)
-            .setModel(getScheme())
-            .setFrom(getFromType())
-            .setTo(getToType());
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/transformer/TransformerDefinition.java b/camel-core/src/main/java/org/apache/camel/model/transformer/TransformerDefinition.java
index 834bd8d..e3d439d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/transformer/TransformerDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/transformer/TransformerDefinition.java
@@ -21,15 +21,13 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.model.InputTypeDefinition;
 import org.apache.camel.model.OutputTypeDefinition;
 import org.apache.camel.spi.DataType;
 import org.apache.camel.spi.Metadata;
-import org.apache.camel.spi.Transformer;
 
 /**
- * <p>Represents a {@link Transformer} which declaratively transforms message content
+ * <p>Represents a {@link org.apache.camel.spi.Transformer} which declaratively transforms message content
  * according to the input type declared by {@link InputTypeDefinition} and/or output type
  * declared by {@link OutputTypeDefinition}.</p>
  * <p>If you specify from='java:com.example.ABC' and to='xml:XYZ', the transformer
@@ -40,7 +38,7 @@ import org.apache.camel.spi.Transformer;
  * Also it's possible to specify scheme='xml' so that the transformer will be picked up
  * for all of java to xml and xml to java transformation.</p>
  * 
- * {@see Transformer}
+ * {@see org.apache.camel.spi.Transformer}
  * {@see InputTypeDefinition}
  * {@see OutputTypeDefinition}
  */
@@ -56,12 +54,6 @@ public abstract class TransformerDefinition {
     @XmlAttribute
     private String toType;
 
-    public Transformer createTransformer(CamelContext context) throws Exception {
-        return doCreateTransformer(context);
-    };
-
-    protected abstract Transformer doCreateTransformer(CamelContext context) throws Exception;
-
     public String getScheme() {
         return scheme;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/validator/CustomValidatorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/validator/CustomValidatorDefinition.java
index befbc02..09c1f2b 100644
--- a/camel-core/src/main/java/org/apache/camel/model/validator/CustomValidatorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/validator/CustomValidatorDefinition.java
@@ -21,7 +21,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Validator;
 
@@ -42,32 +41,6 @@ public class CustomValidatorDefinition extends ValidatorDefinition {
     @XmlAttribute
     private String className;
 
-    @Override
-    protected Validator doCreateValidator(CamelContext context) throws Exception {
-        if (ref == null && className == null) {
-            throw new IllegalArgumentException("'ref' or 'type' must be specified for customValidator");
-        }
-        Validator validator;
-        if (ref != null) {
-            validator = context.getRegistry().lookupByNameAndType(ref, Validator.class);
-            if (validator == null) {
-                throw new IllegalArgumentException("Cannot find validator with ref:" + ref);
-            }
-            if (validator.getType() != null) {
-                throw new IllegalArgumentException(String.format("Validator '%s' is already in use. Please check if duplicate validator exists.", ref));
-            }
-        } else {
-            Class<Validator> validatorClass = context.getClassResolver().resolveMandatoryClass(className, Validator.class);
-            if (validatorClass == null) {
-                throw new IllegalArgumentException("Cannot find validator class: " + className);
-            }
-            validator = context.getInjector().newInstance(validatorClass);
-
-        }
-        validator.setCamelContext(context);
-        return validator.setType(getType());
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/validator/EndpointValidatorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/validator/EndpointValidatorDefinition.java
index 3f5b636..31df8a7 100644
--- a/camel-core/src/main/java/org/apache/camel/model/validator/EndpointValidatorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/validator/EndpointValidatorDefinition.java
@@ -21,11 +21,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
-import org.apache.camel.ExchangePattern;
-import org.apache.camel.impl.validator.ProcessorValidator;
-import org.apache.camel.processor.SendProcessor;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Validator;
 
@@ -33,8 +28,8 @@ import org.apache.camel.spi.Validator;
  * Represents an endpoint {@link Validator} which leverages camel validator component such as
  * <a href="http://camel.apache.org/validation.html">Validator Component</a> and 
  * <a href="http://camel.apache.org/bean-validation.html">Bean Validator Component</a> to
- * perform content validation. A {@link ProcessorValidator} will be created internally
- * with a {@link SendProcessor} which forwards the message to the validator Endpoint.
+ * perform content validation. A {@link org.apache.camel.impl.validator.ProcessorValidator} will be created internally
+ * with a {@link org.apache.camel.processor.SendProcessor} which forwards the message to the validator Endpoint.
  * 
  * {@see ValidatorDefinition}
  * {@see Validator}
@@ -49,16 +44,6 @@ public class EndpointValidatorDefinition extends ValidatorDefinition {
     @XmlAttribute
     private String uri;
 
-    @Override
-    protected Validator doCreateValidator(CamelContext context) throws Exception {
-        Endpoint endpoint = uri != null ? context.getEndpoint(uri)
-            : context.getRegistry().lookupByNameAndType(ref, Endpoint.class);
-        SendProcessor processor = new SendProcessor(endpoint, ExchangePattern.InOut);
-        return new ProcessorValidator(context)
-            .setProcessor(processor)
-            .setType(getType());
-    }
-
     public String getRef() {
         return ref;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/validator/PredicateValidatorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/validator/PredicateValidatorDefinition.java
index ea484df..946917d 100644
--- a/camel-core/src/main/java/org/apache/camel/model/validator/PredicateValidatorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/validator/PredicateValidatorDefinition.java
@@ -21,20 +21,18 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElementRef;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.Expression;
 import org.apache.camel.Predicate;
-import org.apache.camel.impl.validator.ProcessorValidator;
 import org.apache.camel.model.ExpressionNodeHelper;
 import org.apache.camel.model.language.ExpressionDefinition;
-import org.apache.camel.processor.validation.PredicateValidatingProcessor;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.Validator;
 
 /**
  * Represents a predicate {@link Validator} which leverages expression or predicates to
- * perform content validation. A {@link ProcessorValidator} will be created internally
- * with a {@link PredicateValidatingProcessor} which validates the message according to specified expression/predicates.
+ * perform content validation. A {@link org.apache.camel.impl.validator.ProcessorValidator}
+ * will be created internally with a {@link org.apache.camel.processor.validation.PredicateValidatingProcessor}
+ * which validates the message according to specified expression/predicates.
  * 
  * {@see ValidatorDefinition}
  * {@see Validator}
@@ -47,15 +45,6 @@ public class PredicateValidatorDefinition extends ValidatorDefinition {
     @XmlElementRef
     private ExpressionDefinition expression;
 
-    @Override
-    protected Validator doCreateValidator(CamelContext context) throws Exception {
-        Predicate pred = getExpression().createPredicate(context);
-        PredicateValidatingProcessor processor = new PredicateValidatingProcessor(pred);
-        return new ProcessorValidator(context)
-            .setProcessor(processor)
-            .setType(getType());
-    }
-
     public ExpressionDefinition getExpression() {
         return expression;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/model/validator/ValidatorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/validator/ValidatorDefinition.java
index 89a903d..ba75d54 100644
--- a/camel-core/src/main/java/org/apache/camel/model/validator/ValidatorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/validator/ValidatorDefinition.java
@@ -21,7 +21,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.camel.CamelContext;
 import org.apache.camel.model.InputTypeDefinition;
 import org.apache.camel.model.OutputTypeDefinition;
 import org.apache.camel.spi.DataType;
@@ -48,12 +47,6 @@ public abstract class ValidatorDefinition {
     @XmlAttribute
     private String type;
 
-    public Validator createValidator(CamelContext context) throws Exception {
-        return doCreateValidator(context);
-    };
-
-    protected abstract Validator doCreateValidator(CamelContext context) throws Exception;
-
     public String getType() {
         return type;
     }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/AggregateReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/AggregateReifier.java
new file mode 100644
index 0000000..cb9fb40
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/AggregateReifier.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.reifier;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Expression;
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.model.AggregateDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.processor.CamelInternalProcessor;
+import org.apache.camel.processor.aggregate.AggregateController;
+import org.apache.camel.processor.aggregate.AggregateProcessor;
+import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
+import org.apache.camel.spi.AggregationRepository;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.concurrent.SynchronousExecutorService;
+
+class AggregateReifier extends ProcessorReifier<AggregateDefinition> {
+
+    AggregateReifier(ProcessorDefinition<?> definition) {
+        super(AggregateDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return createAggregator(routeContext);
+    }
+
+    protected AggregateProcessor createAggregator(RouteContext routeContext) throws Exception {
+        Processor childProcessor = this.createChildProcessor(routeContext, true);
+
+        // wrap the aggregate route in a unit of work processor
+        CamelInternalProcessor internal = new CamelInternalProcessor(childProcessor);
+        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
+
+        Expression correlation = definition.getExpression().createExpression(routeContext);
+        AggregationStrategy strategy = createAggregationStrategy(routeContext);
+
+        boolean parallel = definition.getParallelProcessing() != null && definition.getParallelProcessing();
+        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, definition, parallel);
+        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "Aggregator", definition, parallel);
+        if (threadPool == null && !parallel) {
+            // executor service is mandatory for the Aggregator
+            // we do not run in parallel mode, but use a synchronous executor, so we run in current thread
+            threadPool = new SynchronousExecutorService();
+            shutdownThreadPool = true;
+        }
+
+        AggregateProcessor answer = new AggregateProcessor(routeContext.getCamelContext(), internal,
+                correlation, strategy, threadPool, shutdownThreadPool);
+
+        AggregationRepository repository = createAggregationRepository(routeContext);
+        if (repository != null) {
+            answer.setAggregationRepository(repository);
+        }
+
+        if (definition.getAggregateController() == null && definition.getAggregateControllerRef() != null) {
+            definition.setAggregateController(routeContext.mandatoryLookup(definition.getAggregateControllerRef(), AggregateController.class));
+        }
+
+        // this EIP supports using a shared timeout checker thread pool or fallback to create a new thread pool
+        boolean shutdownTimeoutThreadPool = false;
+        ScheduledExecutorService timeoutThreadPool = definition.getTimeoutCheckerExecutorService();
+        if (timeoutThreadPool == null && definition.getTimeoutCheckerExecutorServiceRef() != null) {
+            // lookup existing thread pool
+            timeoutThreadPool = routeContext.getCamelContext().getRegistry().lookupByNameAndType(definition.getTimeoutCheckerExecutorServiceRef(), ScheduledExecutorService.class);
+            if (timeoutThreadPool == null) {
+                // then create a thread pool assuming the ref is a thread pool profile id
+                timeoutThreadPool = routeContext.getCamelContext().getExecutorServiceManager().newScheduledThreadPool(this,
+                        AggregateProcessor.AGGREGATE_TIMEOUT_CHECKER, definition.getTimeoutCheckerExecutorServiceRef());
+                if (timeoutThreadPool == null) {
+                    throw new IllegalArgumentException("ExecutorServiceRef " + definition.getTimeoutCheckerExecutorServiceRef()
+                            + " not found in registry (as an ScheduledExecutorService instance) or as a thread pool profile.");
+                }
+                shutdownTimeoutThreadPool = true;
+            }
+        }
+        answer.setTimeoutCheckerExecutorService(timeoutThreadPool);
+        answer.setShutdownTimeoutCheckerExecutorService(shutdownTimeoutThreadPool);
+
+        // set other options
+        answer.setParallelProcessing(parallel);
+        if (definition.getOptimisticLocking() != null) {
+            answer.setOptimisticLocking(definition.getOptimisticLocking());
+        }
+        if (definition.getCompletionPredicate() != null) {
+            Predicate predicate = definition.getCompletionPredicate().createPredicate(routeContext);
+            answer.setCompletionPredicate(predicate);
+        } else if (strategy instanceof Predicate) {
+            // if aggregation strategy implements predicate and was not configured then use as fallback
+            log.debug("Using AggregationStrategy as completion predicate: {}", strategy);
+            answer.setCompletionPredicate((Predicate) strategy);
+        }
+        if (definition.getCompletionTimeoutExpression() != null) {
+            Expression expression = definition.getCompletionTimeoutExpression().createExpression(routeContext);
+            answer.setCompletionTimeoutExpression(expression);
+        }
+        if (definition.getCompletionTimeout() != null) {
+            answer.setCompletionTimeout(definition.getCompletionTimeout());
+        }
+        if (definition.getCompletionInterval() != null) {
+            answer.setCompletionInterval(definition.getCompletionInterval());
+        }
+        if (definition.getCompletionSizeExpression() != null) {
+            Expression expression = definition.getCompletionSizeExpression().createExpression(routeContext);
+            answer.setCompletionSizeExpression(expression);
+        }
+        if (definition.getCompletionSize() != null) {
+            answer.setCompletionSize(definition.getCompletionSize());
+        }
+        if (definition.getCompletionFromBatchConsumer() != null) {
+            answer.setCompletionFromBatchConsumer(definition.getCompletionFromBatchConsumer());
+        }
+        if (definition.getCompletionOnNewCorrelationGroup() != null) {
+            answer.setCompletionOnNewCorrelationGroup(definition.getCompletionOnNewCorrelationGroup());
+        }
+        if (definition.getEagerCheckCompletion() != null) {
+            answer.setEagerCheckCompletion(definition.getEagerCheckCompletion());
+        }
+        if (definition.getIgnoreInvalidCorrelationKeys() != null) {
+            answer.setIgnoreInvalidCorrelationKeys(definition.getIgnoreInvalidCorrelationKeys());
+        }
+        if (definition.getCloseCorrelationKeyOnCompletion() != null) {
+            answer.setCloseCorrelationKeyOnCompletion(definition.getCloseCorrelationKeyOnCompletion());
+        }
+        if (definition.getDiscardOnCompletionTimeout() != null) {
+            answer.setDiscardOnCompletionTimeout(definition.getDiscardOnCompletionTimeout());
+        }
+        if (definition.getForceCompletionOnStop() != null) {
+            answer.setForceCompletionOnStop(definition.getForceCompletionOnStop());
+        }
+        if (definition.getCompleteAllOnStop() != null) {
+            answer.setCompleteAllOnStop(definition.getCompleteAllOnStop());
+        }
+        if (definition.getOptimisticLockRetryPolicy() == null) {
+            if (definition.getOptimisticLockRetryPolicyDefinition() != null) {
+                answer.setOptimisticLockRetryPolicy(definition.getOptimisticLockRetryPolicyDefinition().createOptimisticLockRetryPolicy());
+            }
+        } else {
+            answer.setOptimisticLockRetryPolicy(definition.getOptimisticLockRetryPolicy());
+        }
+        if (definition.getAggregateController() != null) {
+            answer.setAggregateController(definition.getAggregateController());
+        }
+        if (definition.getCompletionTimeoutCheckerInterval() != null) {
+            answer.setCompletionTimeoutCheckerInterval(definition.getCompletionTimeoutCheckerInterval());
+        }
+        return answer;
+    }
+
+    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
+        AggregationStrategy strategy = definition.getAggregationStrategy();
+        if (strategy == null && definition.getStrategyRef() != null) {
+            Object aggStrategy = routeContext.lookup(definition.getStrategyRef(), Object.class);
+            if (aggStrategy instanceof AggregationStrategy) {
+                strategy = (AggregationStrategy) aggStrategy;
+            } else if (aggStrategy != null) {
+                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, definition.getAggregationStrategyMethodName());
+                if (definition.getStrategyMethodAllowNull() != null) {
+                    adapter.setAllowNullNewExchange(definition.getStrategyMethodAllowNull());
+                    adapter.setAllowNullOldExchange(definition.getStrategyMethodAllowNull());
+                }
+                strategy = adapter;
+            } else {
+                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + definition.getStrategyRef());
+            }
+        }
+
+        if (strategy == null) {
+            throw new IllegalArgumentException("AggregationStrategy or AggregationStrategyRef must be set on " + this);
+        }
+
+        if (strategy instanceof CamelContextAware) {
+            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
+        }
+
+        return strategy;
+    }
+
+    private AggregationRepository createAggregationRepository(RouteContext routeContext) {
+        AggregationRepository repository = definition.getAggregationRepository();
+        if (repository == null && definition.getAggregationRepositoryRef() != null) {
+            repository = routeContext.mandatoryLookup(definition.getAggregationRepositoryRef(), AggregationRepository.class);
+        }
+        return repository;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/BeanReifier.java
similarity index 68%
copy from camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/BeanReifier.java
index 9557204..134229e 100644
--- a/camel-core/src/main/java/org/apache/camel/model/BeanDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/BeanReifier.java
@@ -14,13 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlTransient;
+package org.apache.camel.reifier;
 
 import org.apache.camel.Processor;
 import org.apache.camel.RuntimeCamelException;
@@ -32,139 +26,26 @@ import org.apache.camel.component.bean.ConstantStaticTypeBeanHolder;
 import org.apache.camel.component.bean.ConstantTypeBeanHolder;
 import org.apache.camel.component.bean.MethodNotFoundException;
 import org.apache.camel.component.bean.RegistryBean;
-import org.apache.camel.spi.Metadata;
+import org.apache.camel.model.BeanDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.CamelContextHelper;
 import org.apache.camel.util.ObjectHelper;
 
-/**
- * Calls a java bean
- */
-@Metadata(label = "eip,endpoint")
-@XmlRootElement(name = "bean")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
-    @XmlAttribute
-    private String ref;
-    @XmlAttribute
-    private String method;
-    @XmlAttribute
-    private String beanType;
-    @XmlAttribute @Metadata(defaultValue = "true")
-    private Boolean cache;
-    @XmlTransient
-    private Class<?> beanClass;
-    @XmlTransient
-    private Object bean;
-
-    public BeanDefinition() {
-    }
+class BeanReifier extends ProcessorReifier<BeanDefinition> {
 
-    public BeanDefinition(String ref) {
-        this.ref = ref;
-    }
-
-    public BeanDefinition(String ref, String method) {
-        this.ref = ref;
-        this.method = method;
-    }
-
-    @Override
-    public String toString() {
-        return "Bean[" + description() + "]";
-    }
-    
-    public String description() {
-        if (ref != null) {
-            String methodText = "";
-            if (method != null) {
-                methodText = " method:" + method;
-            }
-            return "ref:" + ref + methodText;
-        } else if (bean != null) {
-            return bean.toString();
-        } else if (beanClass != null) {
-            return beanClass.getName();
-        } else if (beanType != null) {
-            return beanType;
-        } else {
-            return "";
-        }
-    }
-    
-    @Override
-    public String getShortName() {
-        return "bean";
+    BeanReifier(ProcessorDefinition<?> definition) {
+        super(BeanDefinition.class.cast(definition));
     }
 
     @Override
-    public String getLabel() {
-        return "bean[" + description() + "]";
-    }
-
-    public String getRef() {
-        return ref;
-    }
-
-    /**
-     * Sets a reference to a bean to use
-     */
-    public void setRef(String ref) {
-        this.ref = ref;
-    }
-
-    public String getMethod() {
-        return method;
-    }
-
-    /**
-     * Sets the method name on the bean to use
-     */
-    public void setMethod(String method) {
-        this.method = method;
-    }
-
-    /**
-     * Sets an instance of the bean to use
-     */
-    public void setBean(Object bean) {
-        this.bean = bean;
-    }
-
-    public String getBeanType() {
-        return beanType;
-    }
-
-    /**
-     * Sets the Class of the bean
-     */
-    public void setBeanType(String beanType) {
-        this.beanType = beanType;
-    }
-
-    /**
-     * Sets the Class of the bean
-     */
-    public void setBeanType(Class<?> beanType) {
-        this.beanClass = beanType;
-    }
-
-    public Boolean getCache() {
-        return cache;
-    }
-
-    /**
-     * Caches the bean lookup, to avoid lookup up bean on every usage.
-     */
-    public void setCache(Boolean cache) {
-        this.cache = cache;
-    }
-
-    // Fluent API
-    //-------------------------------------------------------------------------
-
-    @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Object bean = definition.getBean();
+        String ref = definition.getRef();
+        String method = definition.getMethod();
+        String beanType = definition.getBeanType();
+        Class<?> beanClass = definition.getBeanClass();
+
         BeanProcessor answer;
         Class<?> clazz = bean != null ? bean.getClass() : null;
         BeanHolder beanHolder;
@@ -183,7 +64,7 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
             answer = new BeanProcessor(beanHolder);
         } else {
             if (bean == null) {
-                
+
                 if (beanType == null && beanClass == null) {
                     throw new IllegalArgumentException("bean, ref or beanType must be provided");
                 }
@@ -221,7 +102,7 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
             // to a bean name but the String is being invoke instead
             if (bean instanceof String) {
                 throw new IllegalArgumentException("The bean instance is a java.lang.String type: " + bean
-                    + ". We suppose you want to refer to a bean instance by its id instead. Please use ref.");
+                        + ". We suppose you want to refer to a bean instance by its id instead. Please use ref.");
             }
 
             // the holder should either be bean or type based
@@ -242,7 +123,7 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
             }
             answer = new BeanProcessor(beanHolder);
         }
-        
+
         // check for method exists
         if (method != null) {
             answer.setMethod(method);
@@ -270,7 +151,7 @@ public class BeanDefinition extends NoOutputDefinition<BeanDefinition> {
     }
 
     private boolean isCacheBean() {
-        return cache == null || cache;
+        return definition.getCache() == null || definition.getCache();
     }
 
 }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/CatchReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/CatchReifier.java
new file mode 100644
index 0000000..d14adfa
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/CatchReifier.java
@@ -0,0 +1,81 @@
+/**
+ * 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.reifier;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.model.CatchDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.TryDefinition;
+import org.apache.camel.processor.CatchProcessor;
+import org.apache.camel.spi.RouteContext;
+
+class CatchReifier extends ProcessorReifier<CatchDefinition> {
+
+    CatchReifier(ProcessorDefinition<?> definition) {
+        super(CatchDefinition.class.cast(definition));
+    }
+
+    @Override
+    public CatchProcessor createProcessor(RouteContext routeContext) throws Exception {
+        // create and load exceptions if not done
+        if (definition.getExceptionClasses() == null) {
+            definition.setExceptionClasses(createExceptionClasses(routeContext.getCamelContext()));
+        }
+
+        // must have at least one exception
+        if (definition.getExceptionClasses().isEmpty()) {
+            throw new IllegalArgumentException("At least one Exception must be configured to catch");
+        }
+
+        // parent must be a try
+        if (!(definition.getParent() instanceof TryDefinition)) {
+            throw new IllegalArgumentException("This doCatch should have a doTry as its parent on " + this);
+        }
+
+        // do catch does not mandate a child processor
+        Processor childProcessor = this.createChildProcessor(routeContext, false);
+
+        Predicate when = null;
+        if (definition.getOnWhen() != null) {
+            when = definition.getOnWhen().getExpression().createPredicate(routeContext);
+        }
+
+        Predicate handle = definition.getHandledPolicy();
+        if (definition.getHandled() != null) {
+            handle = definition.getHandled().createPredicate(routeContext);
+        }
+
+        return new CatchProcessor(definition.getExceptionClasses(), childProcessor, when, handle);
+    }
+
+    protected List<Class<? extends Throwable>> createExceptionClasses(CamelContext context) throws ClassNotFoundException {
+        // must use the class resolver from CamelContext to load classes to ensure it can
+        // be loaded in all kind of environments such as JEE servers and OSGi etc.
+        List<String> list = definition.getExceptions();
+        List<Class<? extends Throwable>> answer = new ArrayList<>(list.size());
+        for (String name : list) {
+            Class<Throwable> type = context.getClassResolver().resolveMandatoryClass(name, Throwable.class);
+            answer.add(type);
+        }
+        return answer;
+    }
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ChoiceReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ChoiceReifier.java
new file mode 100644
index 0000000..faccb37
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ChoiceReifier.java
@@ -0,0 +1,64 @@
+/**
+ * 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.reifier;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.Processor;
+import org.apache.camel.model.ChoiceDefinition;
+import org.apache.camel.model.ExpressionNode;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.model.WhenDefinition;
+import org.apache.camel.model.language.ExpressionDefinition;
+import org.apache.camel.processor.ChoiceProcessor;
+import org.apache.camel.processor.FilterProcessor;
+import org.apache.camel.spi.RouteContext;
+
+class ChoiceReifier extends ProcessorReifier<ChoiceDefinition> {
+
+    ChoiceReifier(ProcessorDefinition<?> definition) {
+        super(ChoiceDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        List<FilterProcessor> filters = new ArrayList<>();
+        for (WhenDefinition whenClause : definition.getWhenClauses()) {
+            // also resolve properties and constant fields on embedded expressions in the when clauses
+            ExpressionNode exp = whenClause;
+            ExpressionDefinition expressionDefinition = exp.getExpression();
+            if (expressionDefinition != null) {
+                // resolve properties before we create the processor
+                ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
+
+                // resolve constant fields (eg Exchange.FILE_NAME)
+                ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
+            }
+
+            FilterProcessor filter = (FilterProcessor) createProcessor(routeContext, whenClause);
+            filters.add(filter);
+        }
+        Processor otherwiseProcessor = null;
+        if (definition.getOtherwise() != null) {
+            otherwiseProcessor = createProcessor(routeContext, definition.getOtherwise());
+        }
+        return new ChoiceProcessor(filters, otherwiseProcessor);
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ClaimCheckReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ClaimCheckReifier.java
new file mode 100644
index 0000000..1a38249
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ClaimCheckReifier.java
@@ -0,0 +1,123 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Processor;
+import org.apache.camel.model.ClaimCheckDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.ClaimCheckProcessor;
+import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.support.ObjectHelper;
+
+import static org.apache.camel.util.ObjectHelper.notNull;
+
+class ClaimCheckReifier extends ProcessorReifier<ClaimCheckDefinition> {
+
+    ClaimCheckReifier(ProcessorDefinition<?> definition) {
+        super(ClaimCheckDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        notNull(definition.getOperation(), "operation", this);
+
+        ClaimCheckProcessor claim = new ClaimCheckProcessor();
+        claim.setOperation(definition.getOperation().name());
+        claim.setKey(definition.getKey());
+        claim.setFilter(definition.getFilter());
+
+        AggregationStrategy strategy = createAggregationStrategy(routeContext);
+        if (strategy != null) {
+            claim.setAggregationStrategy(strategy);
+        }
+
+        // only filter or aggregation strategy can be configured not both
+        if (definition.getFilter() != null && strategy != null) {
+            throw new IllegalArgumentException("Cannot use both filter and custom aggregation strategy on ClaimCheck EIP");
+        }
+
+        // validate filter, we cannot have both +/- at the same time
+        if (definition.getFilter() != null) {
+            Iterable it = ObjectHelper.createIterable(definition.getFilter(), ",");
+            boolean includeBody = false;
+            boolean excludeBody = false;
+            for (Object o : it) {
+                String pattern = o.toString();
+                if ("body".equals(pattern) || "+body".equals(pattern)) {
+                    includeBody = true;
+                } else if ("-body".equals(pattern)) {
+                    excludeBody = true;
+                }
+            }
+            if (includeBody && excludeBody) {
+                throw new IllegalArgumentException("Cannot have both include and exclude body at the same time in the filter: " + definition.getFilter());
+            }
+            boolean includeHeaders = false;
+            boolean excludeHeaders = false;
+            for (Object o : it) {
+                String pattern = o.toString();
+                if ("headers".equals(pattern) || "+headers".equals(pattern)) {
+                    includeHeaders = true;
+                } else if ("-headers".equals(pattern)) {
+                    excludeHeaders = true;
+                }
+            }
+            if (includeHeaders && excludeHeaders) {
+                throw new IllegalArgumentException("Cannot have both include and exclude headers at the same time in the filter: " + definition.getFilter());
+            }
+            boolean includeHeader = false;
+            boolean excludeHeader = false;
+            for (Object o : it) {
+                String pattern = o.toString();
+                if (pattern.startsWith("header:") || pattern.startsWith("+header:")) {
+                    includeHeader = true;
+                } else if (pattern.startsWith("-header:")) {
+                    excludeHeader = true;
+                }
+            }
+            if (includeHeader && excludeHeader) {
+                throw new IllegalArgumentException("Cannot have both include and exclude header at the same time in the filter: " + definition.getFilter());
+            }
+        }
+
+        return claim;
+    }
+
+    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
+        AggregationStrategy strategy = definition.getAggregationStrategy();
+        if (strategy == null && definition.getAggregationStrategyRef() != null) {
+            Object aggStrategy = routeContext.lookup(definition.getAggregationStrategyRef(), Object.class);
+            if (aggStrategy instanceof AggregationStrategy) {
+                strategy = (AggregationStrategy) aggStrategy;
+            } else if (aggStrategy != null) {
+                strategy = new AggregationStrategyBeanAdapter(aggStrategy, definition.getAggregationStrategyMethodName());
+            } else {
+                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + definition.getAggregationStrategyRef());
+            }
+        }
+
+        if (strategy instanceof CamelContextAware) {
+            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
+        }
+
+        return strategy;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ConvertBodyReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ConvertBodyReifier.java
new file mode 100644
index 0000000..a9dc65b
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ConvertBodyReifier.java
@@ -0,0 +1,58 @@
+/**
+ * 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.reifier;
+
+import java.nio.charset.Charset;
+import java.nio.charset.UnsupportedCharsetException;
+
+import org.apache.camel.Processor;
+import org.apache.camel.model.ConvertBodyDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.ConvertBodyProcessor;
+import org.apache.camel.spi.RouteContext;
+
+class ConvertBodyReifier extends ProcessorReifier<ConvertBodyDefinition> {
+
+    ConvertBodyReifier(ProcessorDefinition<?> definition) {
+        super(ConvertBodyDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        if (definition.getTypeClass() == null && definition.getType() != null) {
+            definition.setTypeClass(routeContext.getCamelContext().getClassResolver().resolveMandatoryClass(definition.getType()));
+        }
+
+        // validate charset
+        if (definition.getCharset() != null) {
+            validateCharset(definition.getCharset());
+        }
+
+        return new ConvertBodyProcessor(definition.getTypeClass(), definition.getCharset());
+    }
+
+    public static void validateCharset(String charset) throws UnsupportedCharsetException {
+        if (charset != null) {
+            if (Charset.isSupported(charset)) {
+                Charset.forName(charset);
+                return;
+            }
+        }
+        throw new UnsupportedCharsetException(charset);
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/DelayReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/DelayReifier.java
new file mode 100644
index 0000000..645fe67
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/DelayReifier.java
@@ -0,0 +1,66 @@
+/**
+ * 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.reifier;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.model.DelayDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.model.language.ExpressionDefinition;
+import org.apache.camel.processor.Delayer;
+import org.apache.camel.spi.RouteContext;
+
+class DelayReifier extends ExpressionReifier<DelayDefinition> {
+
+    DelayReifier(ProcessorDefinition<?> definition) {
+        super(DelayDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor childProcessor = this.createChildProcessor(routeContext, false);
+        Expression delay = createAbsoluteTimeDelayExpression(routeContext);
+
+        boolean async = definition.getAsyncDelayed() != null && definition.getAsyncDelayed();
+        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, definition, async);
+        ScheduledExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredScheduledExecutorService(routeContext, "Delay", definition, async);
+
+        Delayer answer = new Delayer(routeContext.getCamelContext(), childProcessor, delay, threadPool, shutdownThreadPool);
+        if (definition.getAsyncDelayed() != null) {
+            answer.setAsyncDelayed(definition.getAsyncDelayed());
+        }
+        if (definition.getCallerRunsWhenRejected() == null) {
+            // should be default true
+            answer.setCallerRunsWhenRejected(true);
+        } else {
+            answer.setCallerRunsWhenRejected(definition.getCallerRunsWhenRejected());
+        }
+        return answer;
+    }
+
+    private Expression createAbsoluteTimeDelayExpression(RouteContext routeContext) {
+        ExpressionDefinition expr = definition.getExpression();
+        if (expr != null) {
+            return expr.createExpression(routeContext);
+        }
+        return null;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/DynamicRouterReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/DynamicRouterReifier.java
new file mode 100644
index 0000000..af4dfea
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/DynamicRouterReifier.java
@@ -0,0 +1,59 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.ErrorHandlerFactory;
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.model.DynamicRouterDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.processor.DynamicRouter;
+import org.apache.camel.spi.RouteContext;
+
+class DynamicRouterReifier extends ExpressionReifier<DynamicRouterDefinition<?>> {
+
+    DynamicRouterReifier(ProcessorDefinition<?> definition) {
+        super(DynamicRouterDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Expression expression = definition.getExpression().createExpression(routeContext);
+        String delimiter = definition.getUriDelimiter() != null ? definition.getUriDelimiter() : DynamicRouterDefinition.DEFAULT_DELIMITER;
+
+        DynamicRouter dynamicRouter = new DynamicRouter(routeContext.getCamelContext(), expression, delimiter);
+        if (definition.getIgnoreInvalidEndpoints() != null) {
+            dynamicRouter.setIgnoreInvalidEndpoints(definition.getIgnoreInvalidEndpoints());
+        }
+        if (definition.getCacheSize() != null) {
+            dynamicRouter.setCacheSize(definition.getCacheSize());
+        }
+
+        // and wrap this in an error handler
+        ErrorHandlerFactory builder = ((RouteDefinition) routeContext.getRoute()).getErrorHandlerBuilder();
+        // create error handler (create error handler directly to keep it light weight,
+        // instead of using ProcessorDefinition.wrapInErrorHandler)
+        AsyncProcessor errorHandler = (AsyncProcessor) builder.createErrorHandler(routeContext, dynamicRouter.newRoutingSlipProcessorForErrorHandler());
+        dynamicRouter.setErrorHandler(errorHandler);
+
+        return dynamicRouter;
+    }
+
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/EnrichReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/EnrichReifier.java
new file mode 100644
index 0000000..a9ec09a
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/EnrichReifier.java
@@ -0,0 +1,80 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.model.EnrichDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.Enricher;
+import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
+import org.apache.camel.spi.RouteContext;
+
+class EnrichReifier extends ExpressionReifier<EnrichDefinition> {
+
+    EnrichReifier(ProcessorDefinition<?> definition) {
+        super(EnrichDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+
+        Expression exp = definition.getExpression().createExpression(routeContext);
+        boolean isShareUnitOfWork = definition.getShareUnitOfWork() != null && definition.getShareUnitOfWork();
+        boolean isIgnoreInvalidEndpoint = definition.getIgnoreInvalidEndpoint() != null && definition.getIgnoreInvalidEndpoint();
+
+        Enricher enricher = new Enricher(exp);
+        enricher.setShareUnitOfWork(isShareUnitOfWork);
+        enricher.setIgnoreInvalidEndpoint(isIgnoreInvalidEndpoint);
+        AggregationStrategy strategy = createAggregationStrategy(routeContext);
+        if (strategy != null) {
+            enricher.setAggregationStrategy(strategy);
+        }
+        if (definition.getAggregateOnException() != null) {
+            enricher.setAggregateOnException(definition.getAggregateOnException());
+        }
+        return enricher;
+    }
+
+    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
+        AggregationStrategy strategy = definition.getAggregationStrategy();
+        if (strategy == null && definition.getAggregationStrategyRef() != null) {
+            Object aggStrategy = routeContext.lookup(definition.getAggregationStrategyRef(), Object.class);
+            if (aggStrategy instanceof AggregationStrategy) {
+                strategy = (AggregationStrategy) aggStrategy;
+            } else if (aggStrategy != null) {
+                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, definition.getAggregationStrategyMethodName());
+                if (definition.getAggregationStrategyMethodAllowNull() != null) {
+                    adapter.setAllowNullNewExchange(definition.getAggregationStrategyMethodAllowNull());
+                    adapter.setAllowNullOldExchange(definition.getAggregationStrategyMethodAllowNull());
+                }
+                strategy = adapter;
+            } else {
+                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + definition.getAggregationStrategyRef());
+            }
+        }
+
+        if (strategy instanceof CamelContextAware) {
+            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
+        }
+
+        return strategy;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ExpressionReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ExpressionReifier.java
new file mode 100644
index 0000000..541b20d
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ExpressionReifier.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.reifier;
+
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.model.ExpressionNode;
+import org.apache.camel.processor.FilterProcessor;
+import org.apache.camel.spi.RouteContext;
+
+abstract class ExpressionReifier<T extends ExpressionNode> extends ProcessorReifier<T> {
+
+    protected ExpressionReifier(T definition) {
+        super(definition);
+    }
+
+    /**
+     * Creates the {@link FilterProcessor} from the expression node.
+     *
+     * @param routeContext  the route context
+     * @return the created {@link FilterProcessor}
+     * @throws Exception is thrown if error creating the processor
+     */
+    protected FilterProcessor createFilterProcessor(RouteContext routeContext) throws Exception {
+        Processor childProcessor = createOutputsProcessor(routeContext);
+        return new FilterProcessor(createPredicate(routeContext), childProcessor);
+    }
+
+    /**
+     * Creates the {@link Predicate} from the expression node.
+     *
+     * @param routeContext  the route context
+     * @return the created predicate
+     */
+    protected Predicate createPredicate(RouteContext routeContext) {
+        return definition.getExpression().createPredicate(routeContext);
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/FilterReifier.java
similarity index 53%
copy from camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/FilterReifier.java
index b0a102a..1268705 100644
--- a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/FilterReifier.java
@@ -14,38 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
+package org.apache.camel.reifier;
 
 import org.apache.camel.Processor;
-import org.apache.camel.spi.Metadata;
+import org.apache.camel.model.FilterDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.FilterProcessor;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Routes the message to a sequence of processors.
- */
-@Metadata(label = "eip,routing")
-@XmlRootElement(name = "pipeline")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class PipelineDefinition extends OutputDefinition<PipelineDefinition> {
+class FilterReifier extends ExpressionReifier<FilterDefinition> {
 
-    public PipelineDefinition() {
+    FilterReifier(ProcessorDefinition<?> definition) {
+        super(FilterDefinition.class.cast(definition));
     }
 
     @Override
-    public String getShortName() {
-        return "pipeline";
+    public FilterProcessor createProcessor(RouteContext routeContext) throws Exception {
+        return createFilterProcessor(routeContext);
     }
 
     @Override
-    public String getLabel() {
-        return "pipeline";
+    protected FilterProcessor createFilterProcessor(RouteContext routeContext) throws Exception {
+        // filter EIP should have child outputs
+        Processor childProcessor = this.createChildProcessor(routeContext, true);
+        return new FilterProcessor(createPredicate(routeContext), childProcessor);
     }
 
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return this.createChildProcessor(routeContext, true);
-    }
-}
\ No newline at end of file
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/FinallyReifier.java
similarity index 63%
copy from camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/FinallyReifier.java
index 8e7a943..d3f79f7 100644
--- a/camel-core/src/main/java/org/apache/camel/model/FinallyDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/FinallyReifier.java
@@ -14,48 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
+package org.apache.camel.reifier;
 
 import org.apache.camel.Processor;
+import org.apache.camel.model.FinallyDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.TryDefinition;
 import org.apache.camel.processor.FinallyProcessor;
-import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Path traversed when a try, catch, finally block exits
- */
-@Metadata(label = "error")
-@XmlRootElement(name = "doFinally")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class FinallyDefinition extends OutputDefinition<FinallyDefinition> {
-
-    @Override
-    public String toString() {
-        return "DoFinally[" + getOutputs() + "]";
-    }
-    
-    @Override
-    public String getShortName() {
-        return "doFinally";
-    }
+class FinallyReifier extends ProcessorReifier<FinallyDefinition> {
 
-    @Override
-    public String getLabel() {
-        return "doFinally";
+    FinallyReifier(ProcessorDefinition<?> definition) {
+        super(FinallyDefinition.class.cast(definition));
     }
 
     @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {
         // parent must be a try
-        if (!(getParent() instanceof TryDefinition)) {
+        if (!(definition.getParent() instanceof TryDefinition)) {
             throw new IllegalArgumentException("This doFinally should have a doTry as its parent on " + this);
         }
 
         // do finally does mandate a child processor
         return new FinallyProcessor(this.createChildProcessor(routeContext, false));
     }
+
 }
diff --git a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/HystrixReifier.java
similarity index 62%
copy from components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/HystrixReifier.java
index c3d102e..8aae7a0 100644
--- a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/HystrixReifier.java
@@ -14,20 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.cdi.transaction;
+package org.apache.camel.reifier;
 
-import org.apache.camel.spi.Policy;
+import org.apache.camel.Processor;
+import org.apache.camel.model.HystrixDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Used to expose the method &apos;resolvePolicy&apos; used by
- * {@link JtaTransactionErrorHandlerBuilder} to resolve configured policy
- * references.
- */
-public class TransactedDefinition extends org.apache.camel.model.TransactedDefinition {
+class HystrixReifier extends ProcessorReifier<HystrixDefinition> {
+
+    HystrixReifier(ProcessorDefinition<?> definition) {
+        super(HystrixDefinition.class.cast(definition));
+    }
 
     @Override
-    public Policy resolvePolicy(RouteContext routeContext) {
-        return super.resolvePolicy(routeContext);
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        throw new IllegalStateException("Cannot find camel-hystrix on the classpath.");
     }
+
 }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/IdempotentConsumerReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/IdempotentConsumerReifier.java
new file mode 100644
index 0000000..9b2bc34
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/IdempotentConsumerReifier.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.reifier;
+
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.model.IdempotentConsumerDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.idempotent.IdempotentConsumer;
+import org.apache.camel.spi.IdempotentRepository;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.ObjectHelper;
+
+class IdempotentConsumerReifier extends ExpressionReifier<IdempotentConsumerDefinition> {
+
+    IdempotentConsumerReifier(ProcessorDefinition<?> definition) {
+        super(IdempotentConsumerDefinition.class.cast(definition));
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor childProcessor = this.createChildProcessor(routeContext, true);
+
+        IdempotentRepository<String> idempotentRepository = resolveMessageIdRepository(routeContext);
+        ObjectHelper.notNull(idempotentRepository, "idempotentRepository", this);
+
+        Expression expression = definition.getExpression().createExpression(routeContext);
+
+        // these boolean should be true by default
+        boolean eager = definition.getEager() == null || definition.getEager();
+        boolean duplicate = definition.getSkipDuplicate() == null || definition.getSkipDuplicate();
+        boolean remove = definition.getRemoveOnFailure() == null || definition.getRemoveOnFailure();
+
+        // these boolean should be false by default
+        boolean completionEager = definition.getCompletionEager() != null && definition.getCompletionEager();
+
+        return new IdempotentConsumer(expression, idempotentRepository, eager, completionEager, duplicate, remove, childProcessor);
+    }
+
+    /**
+     * Strategy method to resolve the {@link org.apache.camel.spi.IdempotentRepository} to use
+     *
+     * @param routeContext route context
+     * @return the repository
+     */
+    @SuppressWarnings("unchecked")
+    protected <T> IdempotentRepository<T> resolveMessageIdRepository(RouteContext routeContext) {
+        if (definition.getMessageIdRepositoryRef() != null) {
+            definition.setMessageIdRepository(routeContext.mandatoryLookup(definition.getMessageIdRepositoryRef(), IdempotentRepository.class));
+        }
+        return (IdempotentRepository<T>) definition.getMessageIdRepository();
+    }
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/InterceptFromReifier.java
similarity index 57%
copy from camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/InterceptFromReifier.java
index ffcf3ef..8f222d2 100644
--- a/camel-core/src/main/java/org/apache/camel/model/InterceptFromDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/InterceptFromReifier.java
@@ -14,59 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlRootElement;
+package org.apache.camel.reifier;
 
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
-import org.apache.camel.spi.Metadata;
+import org.apache.camel.model.InterceptFromDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.SetHeaderDefinition;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.ExpressionAdapter;
 
-/**
- * Intercepts incoming messages
- */
-@Metadata(label = "configuration")
-@XmlRootElement(name = "interceptFrom")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class InterceptFromDefinition extends InterceptDefinition {
-    @XmlAttribute
-    protected String uri;
-
-    public InterceptFromDefinition() {
-    }
-
-    public InterceptFromDefinition(String uri) {
-        this.uri = uri;
-    }
-
-    @Override
-    public String toString() {
-        return "InterceptFrom[" + getOutputs() + "]";
-    }
-
-    @Override
-    public String getShortName() {
-        return "interceptFrom";
-    }
-
-    @Override
-    public String getLabel() {
-        return "interceptFrom";
-    }
+class InterceptFromReifier extends InterceptReifier<InterceptFromDefinition> {
 
-    @Override
-    public boolean isAbstract() {
-        return true;
-    }
-
-    @Override
-    public boolean isTopLevelOnly() {
-        return true;
+    InterceptFromReifier(ProcessorDefinition<?> definition) {
+        super((InterceptFromDefinition) definition);
     }
 
     @Override
@@ -87,20 +48,10 @@ public class InterceptFromDefinition extends InterceptDefinition {
                 return "";
             }
         });
-        getOutputs().add(0, headerDefinition);
+        definition.getOutputs().add(0, headerDefinition);
 
         return this.createChildProcessor(routeContext, true);
     }
 
-    public String getUri() {
-        return uri;
-    }
 
-    /**
-     * Intercept incoming messages from the uri or uri pattern.
-     * If this option is not configured, then all incoming messages is intercepted.
-     */
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/InterceptReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/InterceptReifier.java
new file mode 100644
index 0000000..f11a0e6
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/InterceptReifier.java
@@ -0,0 +1,72 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.NamedNode;
+import org.apache.camel.Processor;
+import org.apache.camel.model.InterceptDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.processor.Pipeline;
+import org.apache.camel.spi.InterceptStrategy;
+import org.apache.camel.spi.RouteContext;
+
+class InterceptReifier<T extends InterceptDefinition> extends ProcessorReifier<T> {
+
+    InterceptReifier(ProcessorDefinition<?> definition) {
+        super((T) definition);
+    }
+
+    @Override
+    public Processor createProcessor(final RouteContext routeContext) throws Exception {
+        // create the output processor
+        Processor output = this.createChildProcessor(routeContext, true);
+
+        // add the output as a intercept strategy to the route context so its invoked on each processing step
+        routeContext.getInterceptStrategies().add(new InterceptStrategy() {
+            private Processor interceptedTarget;
+
+            public Processor wrapProcessorInInterceptors(CamelContext context, NamedNode definition,
+                                                         Processor target, Processor nextTarget) throws Exception {
+                // store the target we are intercepting
+                this.interceptedTarget = target;
+
+                // remember the target that was intercepted
+                InterceptReifier.this.definition.getIntercepted().add(interceptedTarget);
+
+                if (interceptedTarget != null) {
+                    // wrap in a pipeline so we continue routing to the next
+                    return Pipeline.newInstance(context, output, interceptedTarget);
+                } else {
+                    return output;
+                }
+            }
+
+            @Override
+            public String toString() {
+                return "intercept[" + (interceptedTarget != null ? interceptedTarget : output) + "]";
+            }
+        });
+
+        // remove me from the route so I am not invoked in a regular route path
+        ((RouteDefinition) routeContext.getRoute()).getOutputs().remove(this);
+        // and return no processor to invoke next from me
+        return null;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/InterceptSendToEndpointReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/InterceptSendToEndpointReifier.java
new file mode 100644
index 0000000..0462fa9
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/InterceptSendToEndpointReifier.java
@@ -0,0 +1,100 @@
+/**
+ * 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.reifier;
+
+import java.util.List;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.InterceptSendToEndpoint;
+import org.apache.camel.model.InterceptSendToEndpointDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.processor.InterceptEndpointProcessor;
+import org.apache.camel.spi.EndpointStrategy;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.support.EndpointHelper;
+import org.apache.camel.util.URISupport;
+
+class InterceptSendToEndpointReifier extends ProcessorReifier<InterceptSendToEndpointDefinition> {
+
+    InterceptSendToEndpointReifier(ProcessorDefinition<?> definition) {
+        super((InterceptSendToEndpointDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(final RouteContext routeContext) throws Exception {
+        // create the detour
+        final Processor detour = this.createChildProcessor(routeContext, true);
+        final String matchURI = definition.getUri();
+
+        // register endpoint callback so we can proxy the endpoint
+        routeContext.getCamelContext().addRegisterEndpointCallback(new EndpointStrategy() {
+            public Endpoint registerEndpoint(String uri, Endpoint endpoint) {
+                if (endpoint instanceof InterceptSendToEndpoint) {
+                    // endpoint already decorated
+                    return endpoint;
+                } else if (matchURI == null || matchPattern(routeContext.getCamelContext(), uri, matchURI)) {
+                    // only proxy if the uri is matched decorate endpoint with our proxy
+                    // should be false by default
+                    boolean skip = definition.getSkipSendToOriginalEndpoint() != null && definition.getSkipSendToOriginalEndpoint();
+                    InterceptSendToEndpoint proxy = new InterceptSendToEndpoint(endpoint, skip);
+                    proxy.setDetour(detour);
+                    return proxy;
+                } else {
+                    // no proxy so return regular endpoint
+                    return endpoint;
+                }
+            }
+        });
+
+
+        // remove the original intercepted route from the outputs as we do not intercept as the regular interceptor
+        // instead we use the proxy endpoints producer do the triggering. That is we trigger when someone sends
+        // an exchange to the endpoint, see InterceptSendToEndpoint for details.
+        RouteDefinition route = (RouteDefinition) routeContext.getRoute();
+        List<ProcessorDefinition<?>> outputs = route.getOutputs();
+        outputs.remove(this);
+
+        return new InterceptEndpointProcessor(matchURI, detour);
+    }
+
+    /**
+     * Does the uri match the pattern.
+     *
+     * @param camelContext the CamelContext
+     * @param uri the uri
+     * @param pattern the pattern, which can be an endpoint uri as well
+     * @return <tt>true</tt> if matched and we should intercept, <tt>false</tt> if not matched, and not intercept.
+     */
+    protected boolean matchPattern(CamelContext camelContext, String uri, String pattern) {
+        // match using the pattern as-is
+        boolean match = EndpointHelper.matchEndpoint(camelContext, uri, pattern);
+        if (!match) {
+            try {
+                // the pattern could be an uri, so we need to normalize it before matching again
+                pattern = URISupport.normalizeUri(pattern);
+                match = EndpointHelper.matchEndpoint(camelContext, uri, pattern);
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+        return match;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/LoadBalanceReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/LoadBalanceReifier.java
new file mode 100644
index 0000000..625dc6f
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/LoadBalanceReifier.java
@@ -0,0 +1,74 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.Processor;
+import org.apache.camel.model.LoadBalanceDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.loadbalancer.FailoverLoadBalancerDefinition;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+import org.apache.camel.reifier.loadbalancer.LoadBalancerReifier;
+import org.apache.camel.spi.RouteContext;
+
+class LoadBalanceReifier extends ProcessorReifier<LoadBalanceDefinition> {
+
+    LoadBalanceReifier(ProcessorDefinition<?> definition) {
+        super((LoadBalanceDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        // the load balancer is stateful so we should only create it once in case its used from a context scoped error handler
+
+        LoadBalancer loadBalancer = definition.getLoadBalancerType().getLoadBalancer();
+        if (loadBalancer == null) {
+            // then create it and reuse it
+            loadBalancer = LoadBalancerReifier.reifier(definition.getLoadBalancerType()).createLoadBalancer(routeContext);
+            definition.getLoadBalancerType().setLoadBalancer(loadBalancer);
+
+            // some load balancer can only support a fixed number of outputs
+            int max = definition.getLoadBalancerType().getMaximumNumberOfOutputs();
+            int size = definition.getOutputs().size();
+            if (size > max) {
+                throw new IllegalArgumentException("To many outputs configured on " + definition.getLoadBalancerType() + ": " + size + " > " + max);
+            }
+
+            for (ProcessorDefinition<?> processorType : definition.getOutputs()) {
+                // output must not be another load balancer
+                // check for instanceof as the code below as there is compilation errors on earlier versions of JDK6
+                // on Windows boxes or with IBM JDKs etc.
+                if (LoadBalanceDefinition.class.isInstance(processorType)) {
+                    throw new IllegalArgumentException("Loadbalancer already configured to: " + definition.getLoadBalancerType() + ". Cannot set it to: " + processorType);
+                }
+                Processor processor = createProcessor(routeContext, processorType);
+                processor = wrapChannel(routeContext, processor, processorType);
+                loadBalancer.addProcessor(processor);
+            }
+        }
+
+        Boolean inherit = definition.isInheritErrorHandler();
+        if (definition.getLoadBalancerType() instanceof FailoverLoadBalancerDefinition) {
+            // special for failover load balancer where you can configure it to not inherit error handler for its children
+            // but the load balancer itself should inherit so Camels error handler can react afterwards
+            inherit = true;
+        }
+        Processor target = wrapChannel(routeContext, loadBalancer, definition, inherit);
+        return target;
+    }
+
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/LogReifier.java
similarity index 50%
copy from camel-core/src/main/java/org/apache/camel/model/LogDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/LogReifier.java
index e6b28d5..cf41dba 100644
--- a/camel-core/src/main/java/org/apache/camel/model/LogDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/LogReifier.java
@@ -14,89 +14,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
+package org.apache.camel.reifier;
 
 import java.util.Map;
 
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlTransient;
-
 import org.apache.camel.Exchange;
 import org.apache.camel.Expression;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.Processor;
+import org.apache.camel.model.Constants;
+import org.apache.camel.model.LogDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.processor.DefaultMaskingFormatter;
 import org.apache.camel.processor.LogProcessor;
+import org.apache.camel.spi.CamelLogger;
 import org.apache.camel.spi.MaskingFormatter;
-import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.support.CamelContextHelper;
-import org.apache.camel.spi.CamelLogger;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.StringHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Logs the defined message to the logger
- */
-@Metadata(label = "eip,configuration")
-@XmlRootElement(name = "log")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class LogDefinition extends NoOutputDefinition<LogDefinition> {
-    @XmlTransient
-    private static final Logger LOG = LoggerFactory.getLogger(LogDefinition.class);
-    @XmlAttribute(required = true)
-    private String message;
-    @XmlAttribute @Metadata(defaultValue = "INFO")
-    private LoggingLevel loggingLevel;
-    @XmlAttribute
-    private String logName;
-    @XmlAttribute
-    private String marker;
-    @XmlAttribute
-    private String loggerRef;
-    @XmlTransient
-    private Logger logger;
+class LogReifier extends ProcessorReifier<LogDefinition> {
 
-    public LogDefinition() {
-    }
-
-    public LogDefinition(String message) {
-        this.message = message;
-    }
-
-    @Override
-    public String toString() {
-        return "Log[" + message + "]";
-    }
-    
-    @Override
-    public String getShortName() {
-        return "log";
-    }
-
-    @Override
-    public String getLabel() {
-        return "log";
+    LogReifier(ProcessorDefinition<?> definition) {
+        super((LogDefinition) definition);
     }
 
     @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {
-        StringHelper.notEmpty(message, "message", this);
+        StringHelper.notEmpty(definition.getMessage(), "message", this);
 
         // use simple language for the message string to give it more power
-        Expression exp = routeContext.getCamelContext().resolveLanguage("simple").createExpression(message);
+        Expression exp = routeContext.getCamelContext().resolveLanguage("simple").createExpression(definition.getMessage());
 
         // get logger explicitely set in the definition
-        Logger logger = this.getLogger();
+        Logger logger = definition.getLogger();
 
         // get logger which may be set in XML definition
-        if (logger == null && ObjectHelper.isNotEmpty(loggerRef)) {
-            logger = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), loggerRef, Logger.class);
+        if (logger == null && ObjectHelper.isNotEmpty(definition.getLoggerRef())) {
+            logger = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), definition.getLoggerRef(), Logger.class);
         }
 
         if (logger == null) {
@@ -104,31 +62,31 @@ public class LogDefinition extends NoOutputDefinition<LogDefinition> {
             Map<String, Logger> availableLoggers = routeContext.lookupByType(Logger.class);
             if (availableLoggers.size() == 1) {
                 logger = availableLoggers.values().iterator().next();
-                LOG.debug("Using custom Logger: {}", logger);
+                log.debug("Using custom Logger: {}", logger);
             } else if (availableLoggers.size() > 1) {
                 // we should log about this somewhere...
-                LOG.debug("More than one {} instance found in the registry. Falling back to create logger by name.", Logger.class.getName());
+                log.debug("More than one {} instance found in the registry. Falling back to create logger by name.", Logger.class.getName());
             }
         }
 
         if (logger == null) {
-            String name = getLogName();
+            String name = definition.getLogName();
             if (name == null) {
                 name = routeContext.getCamelContext().getGlobalOption(Exchange.LOG_EIP_NAME);
                 if (name != null) {
-                    LOG.debug("Using logName from CamelContext properties: {}", name);
+                    log.debug("Using logName from CamelContext properties: {}", name);
                 }
             }
             if (name == null) {
                 name = routeContext.getRoute().getId();
-                LOG.debug("LogName is not configured, using route id as logName: {}", name);
+                log.debug("LogName is not configured, using route id as logName: {}", name);
             }
             logger = LoggerFactory.getLogger(name);
         }
 
         // should be INFO by default
-        LoggingLevel level = getLoggingLevel() != null ? getLoggingLevel() : LoggingLevel.INFO;
-        CamelLogger camelLogger = new CamelLogger(logger, level, getMarker());
+        LoggingLevel level = definition.getLoggingLevel() != null ? definition.getLoggingLevel() : LoggingLevel.INFO;
+        CamelLogger camelLogger = new CamelLogger(logger, level, definition.getMarker());
 
         return new LogProcessor(exp, camelLogger, getMaskingFormatter(routeContext), routeContext.getCamelContext().getLogListeners());
     }
@@ -144,77 +102,4 @@ public class LogDefinition extends NoOutputDefinition<LogDefinition> {
         return null;
     }
 
-    @Override
-    public void addOutput(ProcessorDefinition<?> output) {
-        // add outputs on parent as this log does not support outputs
-        getParent().addOutput(output);
-    }
-
-    public LoggingLevel getLoggingLevel() {
-        return loggingLevel;
-    }
-
-    /**
-     * Sets the logging level.
-     * <p/>
-     * The default value is INFO
-     */
-    public void setLoggingLevel(LoggingLevel loggingLevel) {
-        this.loggingLevel = loggingLevel;
-    }
-
-    public String getMessage() {
-        return message;
-    }
-
-    /**
-     * Sets the log message (uses simple language)
-     */
-    public void setMessage(String message) {
-        this.message = message;
-    }
-
-    public String getLogName() {
-        return logName;
-    }
-
-    /**
-     * Sets the name of the logger
-     */
-    public void setLogName(String logName) {
-        this.logName = logName;
-    }
-
-    public String getMarker() {
-        return marker;
-    }
-
-    /**
-     * To use slf4j marker
-     */
-    public void setMarker(String marker) {
-        this.marker = marker;
-    }
-
-    public String getLoggerRef() {
-        return loggerRef;
-    }
-
-    /**
-     * To refer to a custom logger instance to lookup from the registry.
-     */
-    public void setLoggerRef(String loggerRef) {
-        this.loggerRef = loggerRef;
-    }
-
-    public Logger getLogger() {
-        return logger;
-    }
-
-    /**
-     * To use a custom logger instance
-     */
-    public void setLogger(Logger logger) {
-        this.logger = logger;
-    }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/LoopReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/LoopReifier.java
new file mode 100644
index 0000000..4f5cfcb
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/LoopReifier.java
@@ -0,0 +1,49 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.Expression;
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.model.LoopDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.LoopProcessor;
+import org.apache.camel.spi.RouteContext;
+
+class LoopReifier extends ExpressionReifier<LoopDefinition> {
+
+    LoopReifier(ProcessorDefinition<?> definition) {
+        super((LoopDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor output = this.createChildProcessor(routeContext, true);
+        boolean isCopy = definition.getCopy() != null && definition.getCopy();
+        boolean isWhile = definition.getDoWhile() != null && definition.getDoWhile();
+
+        Predicate predicate = null;
+        Expression expression = null;
+        if (isWhile) {
+            predicate = definition.getExpression().createPredicate(routeContext);
+        } else {
+            expression = definition.getExpression().createExpression(routeContext);
+        }
+        return new LoopProcessor(output, expression, predicate, isCopy);
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/MarshalReifier.java
similarity index 57%
copy from camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/MarshalReifier.java
index b0a102a..924c486 100644
--- a/camel-core/src/main/java/org/apache/camel/model/PipelineDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/MarshalReifier.java
@@ -14,38 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.model;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
+package org.apache.camel.reifier;
 
 import org.apache.camel.Processor;
-import org.apache.camel.spi.Metadata;
+import org.apache.camel.model.DataFormatDefinition;
+import org.apache.camel.model.MarshalDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.MarshalProcessor;
+import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Routes the message to a sequence of processors.
- */
-@Metadata(label = "eip,routing")
-@XmlRootElement(name = "pipeline")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class PipelineDefinition extends OutputDefinition<PipelineDefinition> {
+class MarshalReifier extends ProcessorReifier<MarshalDefinition> {
 
-    public PipelineDefinition() {
+    MarshalReifier(ProcessorDefinition<?> definition) {
+        super((MarshalDefinition) definition);
     }
 
     @Override
-    public String getShortName() {
-        return "pipeline";
+    public Processor createProcessor(RouteContext routeContext) {
+        DataFormat dataFormat = DataFormatDefinition.getDataFormat(routeContext, definition.getDataFormatType(), null);
+        return new MarshalProcessor(dataFormat);
     }
 
-    @Override
-    public String getLabel() {
-        return "pipeline";
-    }
-
-    public Processor createProcessor(RouteContext routeContext) throws Exception {
-        return this.createChildProcessor(routeContext, true);
-    }
-}
\ No newline at end of file
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/MulticastReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/MulticastReifier.java
new file mode 100644
index 0000000..794b073
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/MulticastReifier.java
@@ -0,0 +1,116 @@
+/**
+ * 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.reifier;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Processor;
+import org.apache.camel.model.MulticastDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.processor.MulticastProcessor;
+import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
+import org.apache.camel.processor.aggregate.ShareUnitOfWorkAggregationStrategy;
+import org.apache.camel.processor.aggregate.UseLatestAggregationStrategy;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.support.CamelContextHelper;
+
+class MulticastReifier extends ProcessorReifier<MulticastDefinition> {
+
+    MulticastReifier(ProcessorDefinition<?> definition) {
+        super((MulticastDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Processor answer = this.createChildProcessor(routeContext, true);
+
+        // force the answer as a multicast processor even if there is only one child processor in the multicast
+        if (!(answer instanceof MulticastProcessor)) {
+            List<Processor> list = new ArrayList<>(1);
+            list.add(answer);
+            answer = createCompositeProcessor(routeContext, list);
+        }
+        return answer;
+    }
+
+    protected Processor createCompositeProcessor(RouteContext routeContext, List<Processor> list) throws Exception {
+        final AggregationStrategy strategy = createAggregationStrategy(routeContext);
+
+        boolean isParallelProcessing = definition.getParallelProcessing() != null && definition.getParallelProcessing();
+        boolean isShareUnitOfWork = definition.getShareUnitOfWork() != null && definition.getShareUnitOfWork();
+        boolean isStreaming = definition.getStreaming() != null && definition.getStreaming();
+        boolean isStopOnException = definition.getStopOnException() != null && definition.getStopOnException();
+        boolean isParallelAggregate = definition.getParallelAggregate() != null && definition.getParallelAggregate();
+        boolean isStopOnAggregateException = definition.getStopOnAggregateException() != null && definition.getStopOnAggregateException();
+
+        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, definition, isParallelProcessing);
+        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "Multicast", definition, isParallelProcessing);
+
+        long timeout = definition.getTimeout() != null ? definition.getTimeout() : 0;
+        if (timeout > 0 && !isParallelProcessing) {
+            throw new IllegalArgumentException("Timeout is used but ParallelProcessing has not been enabled.");
+        }
+        if (definition.getOnPrepareRef() != null) {
+            definition.setOnPrepare(CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), definition.getOnPrepareRef(), Processor.class));
+        }
+
+        MulticastProcessor answer = new MulticastProcessor(routeContext.getCamelContext(), list, strategy, isParallelProcessing,
+                threadPool, shutdownThreadPool, isStreaming, isStopOnException, timeout, definition.getOnPrepare(), isShareUnitOfWork, isParallelAggregate, isStopOnAggregateException);
+        return answer;
+    }
+
+    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
+        AggregationStrategy strategy = definition.getAggregationStrategy();
+        if (strategy == null && definition.getStrategyRef() != null) {
+            Object aggStrategy = routeContext.lookup(definition.getStrategyRef(), Object.class);
+            if (aggStrategy instanceof AggregationStrategy) {
+                strategy = (AggregationStrategy) aggStrategy;
+            } else if (aggStrategy != null) {
+                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, definition.getStrategyMethodName());
+                if (definition.getStrategyMethodAllowNull() != null) {
+                    adapter.setAllowNullNewExchange(definition.getStrategyMethodAllowNull());
+                    adapter.setAllowNullOldExchange(definition.getStrategyMethodAllowNull());
+                }
+                strategy = adapter;
+            } else {
+                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + definition.getStrategyRef());
+            }
+        }
+
+        if (strategy == null) {
+            // default to use latest aggregation strategy
+            strategy = new UseLatestAggregationStrategy();
+        }
+
+        if (strategy instanceof CamelContextAware) {
+            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
+        }
+
+        if (definition.getShareUnitOfWork() != null && definition.getShareUnitOfWork()) {
+            // wrap strategy in share unit of work
+            strategy = new ShareUnitOfWorkAggregationStrategy(strategy);
+        }
+
+        return strategy;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/OnCompletionReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/OnCompletionReifier.java
new file mode 100644
index 0000000..a9fd2ac
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/OnCompletionReifier.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.reifier;
+
+import java.util.concurrent.ExecutorService;
+
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.model.OnCompletionDefinition;
+import org.apache.camel.model.OnCompletionMode;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.processor.CamelInternalProcessor;
+import org.apache.camel.processor.OnCompletionProcessor;
+import org.apache.camel.spi.RouteContext;
+
+class OnCompletionReifier extends ProcessorReifier<OnCompletionDefinition> {
+
+    OnCompletionReifier(ProcessorDefinition<?> definition) {
+        super((OnCompletionDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        // assign whether this was a route scoped onCompletion or not
+        // we need to know this later when setting the parent, as only route scoped should have parent
+        // Note: this logic can possible be removed when the Camel routing engine decides at runtime
+        // to apply onCompletion in a more dynamic fashion than current code base
+        // and therefore is in a better position to decide among context/route scoped OnCompletion at runtime
+        Boolean routeScoped = definition.getRouteScoped();
+        if (routeScoped == null) {
+            routeScoped = definition.getParent() != null;
+        }
+
+        boolean isOnCompleteOnly = definition.getOnCompleteOnly() != null && definition.getOnCompleteOnly();
+        boolean isOnFailureOnly = definition.getOnFailureOnly() != null && definition.getOnFailureOnly();
+        boolean isParallelProcessing = definition.getParallelProcessing() != null && definition.getParallelProcessing();
+        boolean original = definition.getUseOriginalMessagePolicy() != null && definition.getUseOriginalMessagePolicy();
+
+        if (isOnCompleteOnly && isOnFailureOnly) {
+            throw new IllegalArgumentException("Both onCompleteOnly and onFailureOnly cannot be true. Only one of them can be true. On node: " + this);
+        }
+        if (original) {
+            // ensure allow original is turned on
+            routeContext.setAllowUseOriginalMessage(true);
+        }
+
+        String routeId = ((RouteDefinition) routeContext.getRoute()).idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
+
+        Processor childProcessor = this.createChildProcessor(routeContext, true);
+
+        // wrap the on completion route in a unit of work processor
+        CamelInternalProcessor internal = new CamelInternalProcessor(childProcessor);
+        internal.addAdvice(new CamelInternalProcessor.UnitOfWorkProcessorAdvice(routeContext));
+
+        definition.setOnCompletion(routeId, internal);
+
+        Predicate when = null;
+        if (definition.getOnWhen() != null) {
+            when = definition.getOnWhen().getExpression().createPredicate(routeContext);
+        }
+
+        boolean shutdownThreadPool = ProcessorDefinitionHelper.willCreateNewThreadPool(routeContext, definition, isParallelProcessing);
+        ExecutorService threadPool = ProcessorDefinitionHelper.getConfiguredExecutorService(routeContext, "OnCompletion", definition, isParallelProcessing);
+
+        // should be after consumer by default
+        boolean afterConsumer = definition.getMode() == null || definition.getMode() == OnCompletionMode.AfterConsumer;
+
+        OnCompletionProcessor answer = new OnCompletionProcessor(routeContext.getCamelContext(), internal,
+                threadPool, shutdownThreadPool, isOnCompleteOnly, isOnFailureOnly, when, original, afterConsumer);
+        return answer;
+    }
+
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/OnExceptionReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/OnExceptionReifier.java
new file mode 100644
index 0000000..8549aca
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/OnExceptionReifier.java
@@ -0,0 +1,164 @@
+/**
+ * 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.reifier;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.camel.Predicate;
+import org.apache.camel.Processor;
+import org.apache.camel.Route;
+import org.apache.camel.builder.ErrorHandlerBuilder;
+import org.apache.camel.model.OnExceptionDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.processor.CatchProcessor;
+import org.apache.camel.processor.FatalFallbackErrorHandler;
+import org.apache.camel.spi.ClassResolver;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.support.CamelContextHelper;
+import org.apache.camel.util.ObjectHelper;
+
+class OnExceptionReifier extends ProcessorReifier<OnExceptionDefinition> {
+
+    OnExceptionReifier(ProcessorDefinition<?> definition) {
+        super((OnExceptionDefinition) definition);
+    }
+
+    @Override
+    public void addRoutes(RouteContext routeContext, Collection<Route> routes) throws Exception {
+        // assign whether this was a route scoped onException or not
+        // we need to know this later when setting the parent, as only route scoped should have parent
+        // Note: this logic can possible be removed when the Camel routing engine decides at runtime
+        // to apply onException in a more dynamic fashion than current code base
+        // and therefore is in a better position to decide among context/route scoped OnException at runtime
+        if (definition.getRouteScoped() == null) {
+            definition.setRouteScoped(definition.getParent() != null);
+        }
+
+        setHandledFromExpressionType(routeContext);
+        setContinuedFromExpressionType(routeContext);
+        setRetryWhileFromExpressionType(routeContext);
+        setOnRedeliveryFromRedeliveryRef(routeContext);
+        setOnExceptionOccurredFromOnExceptionOccurredRef(routeContext);
+
+        // load exception classes
+        if (definition.getExceptions() != null && !definition.getExceptions().isEmpty()) {
+            definition.setExceptionClasses(createExceptionClasses(routeContext.getCamelContext().getClassResolver()));
+        }
+
+        // must validate configuration before creating processor
+        definition.validateConfiguration();
+
+        if (definition.getUseOriginalMessagePolicy() != null && definition.getUseOriginalMessagePolicy()) {
+            // ensure allow original is turned on
+            routeContext.setAllowUseOriginalMessage(true);
+        }
+
+        // lets attach this on exception to the route error handler
+        Processor child = createOutputsProcessor(routeContext);
+        if (child != null) {
+            // wrap in our special safe fallback error handler if OnException have child output
+            Processor errorHandler = new FatalFallbackErrorHandler(child);
+            String id = routeContext.getRoute().getId();
+            definition.setErrorHandler(id, errorHandler);
+        }
+        // lookup the error handler builder
+        ErrorHandlerBuilder builder = (ErrorHandlerBuilder) ((RouteDefinition) routeContext.getRoute()).getErrorHandlerBuilder();
+        // and add this as error handlers
+        builder.addErrorHandlers(routeContext, definition);
+    }
+
+    @Override
+    public CatchProcessor createProcessor(RouteContext routeContext) throws Exception {
+        // load exception classes
+        if (definition.getExceptions() != null && !definition.getExceptions().isEmpty()) {
+            definition.setExceptionClasses(createExceptionClasses(routeContext.getCamelContext().getClassResolver()));
+        }
+
+        if (definition.getUseOriginalMessagePolicy() != null && definition.getUseOriginalMessagePolicy()) {
+            // ensure allow original is turned on
+            routeContext.setAllowUseOriginalMessage(true);
+        }
+
+        // must validate configuration before creating processor
+        definition.validateConfiguration();
+
+        Processor childProcessor = this.createChildProcessor(routeContext, false);
+
+        Predicate when = null;
+        if (definition.getOnWhen() != null) {
+            when = definition.getOnWhen().getExpression().createPredicate(routeContext);
+        }
+
+        Predicate handle = null;
+        if (definition.getHandled() != null) {
+            handle = definition.getHandled().createPredicate(routeContext);
+        }
+
+        return new CatchProcessor(definition.getExceptionClasses(), childProcessor, when, handle);
+    }
+
+    protected List<Class<? extends Throwable>> createExceptionClasses(ClassResolver resolver) throws ClassNotFoundException {
+        List<String> list = definition.getExceptions();
+        List<Class<? extends Throwable>> answer = new ArrayList<>(list.size());
+        for (String name : list) {
+            Class<? extends Throwable> type = resolver.resolveMandatoryClass(name, Throwable.class);
+            answer.add(type);
+        }
+        return answer;
+    }
+
+    private void setHandledFromExpressionType(RouteContext routeContext) {
+        if (definition.getHandled() != null && definition.getHandledPolicy() == null && routeContext != null) {
+            definition.handled(definition.getHandled().createPredicate(routeContext));
+        }
+    }
+
+    private void setContinuedFromExpressionType(RouteContext routeContext) {
+        if (definition.getContinued() != null && definition.getContinuedPolicy() == null && routeContext != null) {
+            definition.continued(definition.getContinued().createPredicate(routeContext));
+        }
+    }
+
+    private void setRetryWhileFromExpressionType(RouteContext routeContext) {
+        if (definition.getRetryWhile() != null && definition.getRetryWhilePolicy() == null && routeContext != null) {
+            definition.retryWhile(definition.getRetryWhile().createPredicate(routeContext));
+        }
+    }
+
+    private void setOnRedeliveryFromRedeliveryRef(RouteContext routeContext) {
+        // lookup onRedelivery if ref is provided
+        if (ObjectHelper.isNotEmpty(definition.getOnRedeliveryRef())) {
+            // if ref is provided then use mandatory lookup to fail if not found
+            Processor onRedelivery = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), definition.getOnRedeliveryRef(), Processor.class);
+            definition.setOnRedelivery(onRedelivery);
+        }
+    }
+
+    private void setOnExceptionOccurredFromOnExceptionOccurredRef(RouteContext routeContext) {
+        // lookup onRedelivery if ref is provided
+        if (ObjectHelper.isNotEmpty(definition.getOnExceptionOccurredRef())) {
+            // if ref is provided then use mandatory lookup to fail if not found
+            Processor onExceptionOccurred = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), definition.getOnExceptionOccurredRef(), Processor.class);
+            definition.setOnExceptionOccurred(onExceptionOccurred);
+        }
+    }
+
+
+}
diff --git a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/OnFallbackReifier.java
similarity index 63%
copy from components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/OnFallbackReifier.java
index c3d102e..2db41fb 100644
--- a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/OnFallbackReifier.java
@@ -14,20 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.cdi.transaction;
+package org.apache.camel.reifier;
 
-import org.apache.camel.spi.Policy;
+import org.apache.camel.Processor;
+import org.apache.camel.model.OnFallbackDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Used to expose the method &apos;resolvePolicy&apos; used by
- * {@link JtaTransactionErrorHandlerBuilder} to resolve configured policy
- * references.
- */
-public class TransactedDefinition extends org.apache.camel.model.TransactedDefinition {
+class OnFallbackReifier extends ProcessorReifier<OnFallbackDefinition> {
+
+    OnFallbackReifier(ProcessorDefinition<?> definition) {
+        super((OnFallbackDefinition) definition);
+    }
 
     @Override
-    public Policy resolvePolicy(RouteContext routeContext) {
-        return super.resolvePolicy(routeContext);
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return this.createChildProcessor(routeContext, false);
     }
+
 }
diff --git a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/OtherwiseReifier.java
similarity index 62%
copy from components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/OtherwiseReifier.java
index c3d102e..8696aee 100644
--- a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/OtherwiseReifier.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.cdi.transaction;
+package org.apache.camel.reifier;
 
-import org.apache.camel.spi.Policy;
+import org.apache.camel.Processor;
+import org.apache.camel.model.OtherwiseDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Used to expose the method &apos;resolvePolicy&apos; used by
- * {@link JtaTransactionErrorHandlerBuilder} to resolve configured policy
- * references.
- */
-public class TransactedDefinition extends org.apache.camel.model.TransactedDefinition {
+class OtherwiseReifier extends ProcessorReifier<OtherwiseDefinition> {
+
+    OtherwiseReifier(ProcessorDefinition<?> definition) {
+        super(OtherwiseDefinition.class.cast(definition));
+    }
 
     @Override
-    public Policy resolvePolicy(RouteContext routeContext) {
-        return super.resolvePolicy(routeContext);
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return this.createChildProcessor(routeContext, false);
     }
 }
diff --git a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java b/camel-core/src/main/java/org/apache/camel/reifier/PipelineReifier.java
similarity index 63%
copy from components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
copy to camel-core/src/main/java/org/apache/camel/reifier/PipelineReifier.java
index c3d102e..79f2df8 100644
--- a/components/camel-cdi/src/main/java/org/apache/camel/cdi/transaction/TransactedDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/reifier/PipelineReifier.java
@@ -14,20 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.cdi.transaction;
+package org.apache.camel.reifier;
 
-import org.apache.camel.spi.Policy;
+import org.apache.camel.Processor;
+import org.apache.camel.model.PipelineDefinition;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.RouteContext;
 
-/**
- * Used to expose the method &apos;resolvePolicy&apos; used by
- * {@link JtaTransactionErrorHandlerBuilder} to resolve configured policy
- * references.
- */
-public class TransactedDefinition extends org.apache.camel.model.TransactedDefinition {
+class PipelineReifier extends ProcessorReifier<PipelineDefinition> {
+
+    PipelineReifier(ProcessorDefinition<?> definition) {
+        super(PipelineDefinition.class.cast(definition));
+    }
 
     @Override
-    public Policy resolvePolicy(RouteContext routeContext) {
-        return super.resolvePolicy(routeContext);
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        return this.createChildProcessor(routeContext, true);
     }
 }
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/PolicyReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/PolicyReifier.java
new file mode 100644
index 0000000..2f58708
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/PolicyReifier.java
@@ -0,0 +1,63 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.Processor;
+import org.apache.camel.Service;
+import org.apache.camel.model.PolicyDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.WrapProcessor;
+import org.apache.camel.spi.Policy;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.ObjectHelper;
+
+class PolicyReifier extends ProcessorReifier<PolicyDefinition> {
+
+    PolicyReifier(ProcessorDefinition<?> definition) {
+        super((PolicyDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+        Policy policy = resolvePolicy(routeContext);
+        ObjectHelper.notNull(policy, "policy", definition);
+
+        // before wrap
+        policy.beforeWrap(routeContext, definition);
+
+        // create processor after the before wrap
+        Processor childProcessor = this.createChildProcessor(routeContext, true);
+
+        // wrap
+        Processor target = policy.wrap(routeContext, childProcessor);
+
+        if (!(target instanceof Service)) {
+            // wrap the target so it becomes a service and we can manage its lifecycle
+            target = new WrapProcessor(target, childProcessor);
+        }
+        return target;
+    }
+
+    protected Policy resolvePolicy(RouteContext routeContext) {
+        if (definition.getPolicy() != null) {
+            return definition.getPolicy();
+        }
+        // reuse code on transacted definition to do the resolution
+        return TransactedReifier.resolvePolicy(routeContext, definition.getRef(), definition.getType());
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/PollEnrichReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/PollEnrichReifier.java
new file mode 100644
index 0000000..82fd9a7
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/PollEnrichReifier.java
@@ -0,0 +1,87 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.model.PollEnrichDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.PollEnricher;
+import org.apache.camel.processor.aggregate.AggregationStrategy;
+import org.apache.camel.processor.aggregate.AggregationStrategyBeanAdapter;
+import org.apache.camel.spi.RouteContext;
+
+class PollEnrichReifier extends ProcessorReifier<PollEnrichDefinition> {
+
+    PollEnrichReifier(ProcessorDefinition<?> definition) {
+        super((PollEnrichDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) throws Exception {
+
+        // if no timeout then we should block, and there use a negative timeout
+        long time = definition.getTimeout() != null ? definition.getTimeout() : -1;
+        boolean isIgnoreInvalidEndpoint = definition.getIgnoreInvalidEndpoint() != null && definition.getIgnoreInvalidEndpoint();
+        Expression exp = definition.getExpression().createExpression(routeContext);
+
+        PollEnricher enricher = new PollEnricher(exp, time);
+
+        AggregationStrategy strategy = createAggregationStrategy(routeContext);
+        if (strategy == null) {
+            enricher.setDefaultAggregationStrategy();
+        } else {
+            enricher.setAggregationStrategy(strategy);
+        }
+        if (definition.getAggregateOnException() != null) {
+            enricher.setAggregateOnException(definition.getAggregateOnException());
+        }
+        if (definition.getCacheSize() != null) {
+            enricher.setCacheSize(definition.getCacheSize());
+        }
+        enricher.setIgnoreInvalidEndpoint(isIgnoreInvalidEndpoint);
+
+        return enricher;
+    }
+
+    private AggregationStrategy createAggregationStrategy(RouteContext routeContext) {
+        AggregationStrategy strategy = definition.getAggregationStrategy();
+        if (strategy == null && definition.getAggregationStrategyRef() != null) {
+            Object aggStrategy = routeContext.lookup(definition.getAggregationStrategyRef(), Object.class);
+            if (aggStrategy instanceof AggregationStrategy) {
+                strategy = (AggregationStrategy) aggStrategy;
+            } else if (aggStrategy != null) {
+                AggregationStrategyBeanAdapter adapter = new AggregationStrategyBeanAdapter(aggStrategy, definition.getAggregationStrategyMethodName());
+                if (definition.getAggregationStrategyMethodAllowNull() != null) {
+                    adapter.setAllowNullNewExchange(definition.getAggregationStrategyMethodAllowNull());
+                    adapter.setAllowNullOldExchange(definition.getAggregationStrategyMethodAllowNull());
+                }
+                strategy = adapter;
+            } else {
+                throw new IllegalArgumentException("Cannot find AggregationStrategy in Registry with name: " + definition.getAggregationStrategyRef());
+            }
+        }
+
+        if (strategy instanceof CamelContextAware) {
+            ((CamelContextAware) strategy).setCamelContext(routeContext.getCamelContext());
+        }
+
+        return strategy;
+    }
+
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ProcessReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ProcessReifier.java
new file mode 100644
index 0000000..350bb74
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ProcessReifier.java
@@ -0,0 +1,56 @@
+/**
+ * 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.reifier;
+
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.Processor;
+import org.apache.camel.Service;
+import org.apache.camel.model.ProcessDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.DelegateAsyncProcessor;
+import org.apache.camel.processor.DelegateSyncProcessor;
+import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.ObjectHelper;
+
+class ProcessReifier extends ProcessorReifier<ProcessDefinition> {
+
+    ProcessReifier(ProcessorDefinition<?> definition) {
+        super((ProcessDefinition) definition);
+    }
+
+    @Override
+    public Processor createProcessor(RouteContext routeContext) {
+        Processor answer = definition.getProcessor();
+        if (answer == null) {
+            ObjectHelper.notNull(definition.getRef(), "ref", definition);
+            answer = routeContext.mandatoryLookup(definition.getRef(), Processor.class);
+        }
+
+        // ensure its wrapped in a Service so we can manage it from eg. JMX
+        // (a Processor must be a Service to be enlisted in JMX)
+        if (!(answer instanceof Service)) {
+            if (answer instanceof AsyncProcessor) {
+                // the processor is async by nature so use the async delegate
+                answer = new DelegateAsyncProcessor(answer);
+            } else {
+                // the processor is sync by nature so use the sync delegate
+                answer = new DelegateSyncProcessor(answer);
+            }
+        }
+        return answer;
+    }
+}
diff --git a/camel-core/src/main/java/org/apache/camel/reifier/ProcessorReifier.java b/camel-core/src/main/java/org/apache/camel/reifier/ProcessorReifier.java
new file mode 100644
index 0000000..2b788cd
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/reifier/ProcessorReifier.java
@@ -0,0 +1,579 @@
+/**
+ * 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.reifier;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.camel.Channel;
+import org.apache.camel.ErrorHandlerFactory;
+import org.apache.camel.Processor;
+import org.apache.camel.Route;
+import org.apache.camel.model.AggregateDefinition;
+import org.apache.camel.model.BeanDefinition;
+import org.apache.camel.model.CatchDefinition;
+import org.apache.camel.model.ChoiceDefinition;
+import org.apache.camel.model.ClaimCheckDefinition;
+import org.apache.camel.model.ConvertBodyDefinition;
+import org.apache.camel.model.DelayDefinition;
+import org.apache.camel.model.DynamicRouterDefinition;
+import org.apache.camel.model.EnrichDefinition;
+import org.apache.camel.model.ExpressionNode;
+import org.apache.camel.model.FilterDefinition;
+import org.apache.camel.model.FinallyDefinition;
+import org.apache.camel.model.HystrixDefinition;
+import org.apache.camel.model.IdempotentConsumerDefinition;
+import org.apache.camel.model.InOnlyDefinition;
+import org.apache.camel.model.InOutDefinition;
+import org.apache.camel.model.InterceptDefinition;
+import org.apache.camel.model.InterceptFromDefinition;
+import org.apache.camel.model.InterceptSendToEndpointDefinition;
+import org.apache.camel.model.LoadBalanceDefinition;
+import org.apache.camel.model.LogDefinition;
+import org.apache.camel.model.LoopDefinition;
+import org.apache.camel.model.MarshalDefinition;
+import org.apache.camel.model.ModelChannel;
+import org.apache.camel.model.MulticastDefinition;
+import org.apache.camel.model.OnCompletionDefinition;
+import org.apache.camel.model.OnExceptionDefinition;
+import org.apache.camel.model.OnFallbackDefinition;
+import org.apache.camel.model.OtherwiseDefinition;
+import org.apache.camel.model.PipelineDefinition;
+import org.apache.camel.model.PolicyDefinition;
+import org.apache.camel.model.PollEnrichDefinition;
+import org.apache.camel.model.ProcessDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ProcessorDefinitionHelper;
+import org.apache.camel.model.RecipientListDefinition;
+import org.apache.camel.model.RemoveHeaderDefinition;
+import org.apache.camel.model.RemoveHeadersDefinition;
+import org.apache.camel.model.RemovePropertiesDefinition;
+import org.apache.camel.model.RemovePropertyDefinition;
+import org.apache.camel.model.ResequenceDefinition;
+import org.apache.camel.model.RethrowDefinition;
+import org.apache.camel.model.RollbackDefinition;
+import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.model.RoutingSlipDefinition;
+import org.apache.camel.model.SagaDefinition;
+import org.apache.camel.model.SamplingDefinition;
+import org.apache.camel.model.ScriptDefinition;
+import org.apache.camel.model.SetBodyDefinition;
+import org.apache.camel.model.SetExchangePatternDefinition;
+import org.apache.camel.model.SetFaultBodyDefinition;
+import org.apache.camel.model.SetHeaderDefinition;
+import org.apache.camel.model.SetPropertyDefinition;
+import org.apache.camel.model.SortDefinition;
+import org.apache.camel.model.SplitDefinition;
+import org.apache.camel.model.StopDefinition;
+import org.apache.camel.model.ThreadsDefinition;
+import org.apache.camel.model.ThrottleDefinition;
+import org.apache.camel.model.ThrowExceptionDefinition;
+import org.apache.camel.model.ToDefinition;
+import org.apache.camel.model.ToDynamicDefinition;
+import org.apache.camel.model.TransactedDefinition;
+import org.apache.camel.model.TransformDefinition;
+import org.apache.camel.model.TryDefinition;
+import org.apache.camel.model.UnmarshalDefinition;
+import org.apache.camel.model.ValidateDefinition;
+import org.apache.camel.model.WhenDefinition;
+import org.apache.camel.model.WhenSkipSendToEndpointDefinition;
+import org.apache.camel.model.WireTapDefinition;
+import org.apache.camel.model.cloud.ServiceCallDefinition;
+import org.apache.camel.model.language.ExpressionDefinition;
+import org.apache.camel.processor.InterceptEndpointProcessor;
+import org.apache.camel.processor.Pipeline;
+import org.apache.camel.processor.interceptor.DefaultChannel;
+import org.apache.camel.processor.interceptor.HandleFault;
+import org.apache.camel.spi.IdAware;
+import org.apache.camel.spi.InterceptStrategy;
+import org.apache.camel.spi.LifecycleStrategy;
+import org.apache.camel.spi.RouteContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class ProcessorReifier<T extends ProcessorDefinition<?>> {
+
+    private static final Map<Class<?>, Function<ProcessorDefinition<?>, ProcessorReifier<? extends ProcessorDefinition<?>>>> PROCESSORS;
+    static {
+        Map<Class<?>, Function<ProcessorDefinition<?>, ProcessorReifier<? extends ProcessorDefinition<?>>>> map = new HashMap<>();
+        map.put(AggregateDefinition.class, AggregateReifier::new);
+        map.put(BeanDefinition.class, BeanReifier::new);
+        map.put(CatchDefinition.class, CatchReifier::new);
+        map.put(ChoiceDefinition.class, ChoiceReifier::new);
+        map.put(ClaimCheckDefinition.class, ClaimCheckReifier::new);
+        map.put(ConvertBodyDefinition.class, ConvertBodyReifier::new);
+        map.put(DelayDefinition.class, DelayReifier::new);
+        map.put(DynamicRouterDefinition.class, DynamicRouterReifier::new);
+        map.put(EnrichDefinition.class, EnrichReifier::new);
+        map.put(FilterDefinition.class, FilterReifier::new);
+        map.put(FinallyDefinition.class, FinallyReifier::new);
+        map.put(HystrixDefinition.class, HystrixReifier::new);
+        map.put(IdempotentConsumerDefinition.class, IdempotentConsumerReifier::new);
+        map.put(InOnlyDefinition.class, SendReifier::new);
+        map.put(InOutDefinition.class, SendReifier::new);
+        map.put(InterceptDefinition.class, InterceptReifier::new);
+        map.put(InterceptFromDefinition.class, InterceptFromReifier::new);
+        map.put(InterceptSendToEndpointDefinition.class, InterceptSendToEndpointReifier::new);
+        map.put(LoadBalanceDefinition.class, LoadBalanceReifier::new);
+        map.put(LogDefinition.class, LogReifier::new);
+        map.put(LoopDefinition.class, LoopReifier::new);
+        map.put(MarshalDefinition.class, MarshalReifier::new);
+        map.put(MulticastDefinition.class, MulticastReifier::new);
+        map.put(OnCompletionDefinition.class, OnCompletionReifier::new);
+        map.put(OnExceptionDefinition.class, OnExceptionReifier::new);
+        map.put(OnFallbackDefinition.class, OnFallbackReifier::new);
+        map.put(OtherwiseDefinition.class, OtherwiseReifier::new);
+        map.put(PipelineDefinition.class, PipelineReifier::new);
+        map.put(PolicyDefinition.class, PolicyReifier::new);
+        map.put(PollEnrichDefinition.class, PollEnrichReifier::new);
+        map.put(ProcessDefinition.class, ProcessReifier::new);
+        map.put(RecipientListDefinition.class, RecipientListReifier::new);
+        map.put(RemoveHeaderDefinition.class, RemoveHeaderReifier::new);
+        map.put(RemoveHeadersDefinition.class, RemoveHeadersReifier::new);
+        map.put(RemovePropertiesDefinition.class, RemovePropertiesReifier::new);
+        map.put(RemovePropertyDefinition.class, RemovePropertyReifier::new);
+        map.put(ResequenceDefinition.class, ResequenceReifier::new);
+        map.put(RethrowDefinition.class, RethrowReifier::new);
+        map.put(RollbackDefinition.class, RollbackReifier::new);
+        map.put(RouteDefinition.class, RouteReifier::new);
+        map.put(RoutingSlipDefinition.class, RoutingSlipReifier::new);
+        map.put(SagaDefinition.class, SagaReifier::new);
+        map.put(SamplingDefinition.class, SamplingReifier::new);
+        map.put(ScriptDefinition.class, ScriptReifier::new);
+        map.put(ServiceCallDefinition.class, ServiceCallReifier::new);
+        map.put(SetBodyDefinition.class, SetBodyReifier::new);
+        map.put(SetExchangePatternDefinition.class, SetExchangePatternReifier::new);
+        map.put(SetFaultBodyDefinition.class, SetFaultBodyReifier::new);
+        map.put(SetHeaderDefinition.class, SetHeaderReifier::new);
+        map.put(SetPropertyDefinition.class, SetPropertyReifier::new);
+        map.put(SortDefinition.class, SortReifier::new);
+        map.put(SplitDefinition.class, SplitReifier::new);
+        map.put(StopDefinition.class, StopReifier::new);
+        map.put(ThreadsDefinition.class, ThreadsReifier::new);
+        map.put(ThrottleDefinition.class, ThrottleReifier::new);
+        map.put(ThrowExceptionDefinition.class, ThrowExceptionReifier::new);
+        map.put(ToDefinition.class, SendReifier::new);
+        map.put(ToDynamicDefinition.class, ToDynamicReifier::new);
+        map.put(TransactedDefinition.class, TransactedReifier::new);
+        map.put(TransformDefinition.class, TransformReifier::new);
+        map.put(TryDefinition.class, TryReifier::new);
+        map.put(UnmarshalDefinition.class, UnmarshalReifier::new);
+        map.put(ValidateDefinition.class, ValidateReifier::new);
+        map.put(WireTapDefinition.class, WireTapReifier::new);
+        map.put(WhenSkipSendToEndpointDefinition.class, WhenSkipSendToEndpointReifier::new);
+        map.put(WhenDefinition.class, WhenReifier::new);
+        PROCESSORS = map;
+    }
+
+    public static ProcessorReifier<? extends ProcessorDefinition<?>> reifier(ProcessorDefinition<?> definition) {
+        Function<ProcessorDefinition<?>, ProcessorReifier<? extends ProcessorDefinition<?>>> reifier = PROCESSORS.get(definition.getClass());
+        if (reifier != null) {
+            return reifier.apply(definition);
+        }
+        throw new IllegalStateException("Unsupported definition: " + definition);
+    }
+
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
+    protected final T definition;
+
+    public ProcessorReifier(T definition) {
+        this.definition = definition;
+    }
+
+    /**
+     * Override this in definition class and implement logic to create the processor
+     * based on the definition model.
+     */
+    public abstract Processor createProcessor(RouteContext routeContext) throws Exception;
+
+    /**
+     * Prefer to use {#link #createChildProcessor}.
+     */
+    protected Processor createOutputsProcessor(RouteContext routeContext) throws Exception {
+        Collection<ProcessorDefinition<?>> outputs = definition.getOutputs();
+        return createOutputsProcessor(routeContext, outputs);
+    }
+
+    /**
+     * Creates the child processor (outputs) from the current definition
+     *
+     * @param routeContext   the route context
+     * @param mandatory      whether or not children is mandatory (ie the definition should have outputs)
+     * @return the created children, or <tt>null</tt> if definition had no output
+     * @throws Exception is thrown if error creating the child or if it was mandatory and there was no output defined on definition
+     */
+    protected Processor createChildProcessor(RouteContext routeContext, boolean mandatory) throws Exception {
+        Processor children = null;
+        // at first use custom factory
+        if (routeContext.getCamelContext().getProcessorFactory() != null) {
+            children = routeContext.getCamelContext().getProcessorFactory().createChildProcessor(routeContext, definition, mandatory);
+        }
+        // fallback to default implementation if factory did not create the child
+        if (children == null) {
+            children = createOutputsProcessor(routeContext);
+        }
+
+        if (children == null && mandatory) {
+            throw new IllegalArgumentException("Definition has no children on " + this);
+        }
+        return children;
+    }
+
+    public void addRoutes(RouteContext routeContext, Collection<Route> routes) throws Exception {
+        Processor processor = makeProcessor(routeContext);
+        if (processor == null) {
+            // no processor to add
+            return;
+        }
+
+        if (!routeContext.isRouteAdded()) {
+            boolean endpointInterceptor = false;
+
+            // are we routing to an endpoint interceptor, if so we should not add it as an event driven
+            // processor as we use the producer to trigger the interceptor
+            if (processor instanceof Channel) {
+                Channel channel = (Channel) processor;
+                Processor next = channel.getNextProcessor();
+                if (next instanceof InterceptEndpointProcessor) {
+                    endpointInterceptor = true;
+                }
+            }
+
+            // only add regular processors as event driven
+            if (endpointInterceptor) {
+                log.debug("Endpoint interceptor should not be added as an event driven consumer route: {}", processor);
+            } else {
+                log.trace("Adding event driven processor: {}", processor);
+                routeContext.addEventDrivenProcessor(processor);
+            }
+
+        }
+    }
+
+    /**
+     * Wraps the child processor in whatever necessary interceptors and error handlers
+     */
+    public Processor wrapProcessor(RouteContext routeContext, Processor processor) throws Exception {
+        // dont double wrap
+        if (processor instanceof Channel) {
+            return processor;
+        }
+        return wrapChannel(routeContext, processor, null);
+    }
+
+    protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child) throws Exception {
+        return wrapChannel(routeContext, processor, child, definition.isInheritErrorHandler());
+    }
+
+    protected Processor wrapChannel(RouteContext routeContext, Processor processor, ProcessorDefinition<?> child, Boolean inheritErrorHandler) throws Exception {
+        // put a channel in between this and each output to control the route flow logic
+        ModelChannel channel = createChannel(routeContext);
+        channel.setNextProcessor(processor);
+
+        // add interceptor strategies to the channel must be in this order: camel context, route context, local
+        addInterceptStrategies(routeContext, channel, routeContext.getCamelContext().getInterceptStrategies());
+        addInterceptStrategies(routeContext, channel, routeContext.getInterceptStrategies());
+        addInterceptStrategies(routeContext, channel, definition.getInterceptStrategies());
+
+        // set the child before init the channel
+        channel.setChildDefinition(child);
+        channel.initChannel(definition, routeContext);
+
+        // set the error handler, must be done after init as we can set the error handler as first in the chain
+        if (definition instanceof TryDefinition || definition instanceof CatchDefinition || definition instanceof FinallyDefinition) {
+            // do not use error handler for try .. catch .. finally blocks as it will handle errors itself
+            log.trace("{} is part of doTry .. doCatch .. doFinally so no error handler is applied", definition);
+        } else if (ProcessorDefinitionHelper.isParentOfType(TryDefinition.class, definition, true)
+                || ProcessorDefinitionHelper.isParentOfType(CatchDefinition.class, definition, true)
+                || ProcessorDefinitionHelper.isParentOfType(FinallyDefinition.class, definition, true)) {
+            // do not use error handler for try .. catch .. finally blocks as it will handle errors itself
+            // by checking that any of our parent(s) is not a try .. catch or finally type
+            log.trace("{} is part of doTry .. doCatch .. doFinally so no error handler is applied", definition);
+        } else if (definition instanceof OnExceptionDefinition || ProcessorDefinitionHelper.isParentOfType(OnExceptionDefinition.class, definition, true)) {
+            log.trace("{} is part of OnException so no error handler is applied", definition);
+            // do not use error handler for onExceptions blocks as it will handle errors itself
+        } else if (definition instanceof HystrixDefinition || ProcessorDefinitionHelper.isParentOfType(HystrixDefinition.class, definition, true)) {
+            // do not use error handler for hystrix as it offers circuit breaking with fallback for its outputs
+            // however if inherit error handler is enabled, we need to wrap an error handler on the hystrix parent
+            if (inheritErrorHandler != null && inheritErrorHandler && child == null) {
+                // only wrap the parent (not the children of the hystrix)
+                wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
+            } else {
+                log.trace("{} is part of HystrixCircuitBreaker so no error handler is applied", definition);
+            }
+        } else if (definition instanceof MulticastDefinition) {
+            // do not use error handler for multicast as it offers fine grained error handlers for its outputs
+            // however if share unit of work is enabled, we need to wrap an error handler on the multicast parent
+            MulticastDefinition def = (MulticastDefinition) definition;
+            boolean isShareUnitOfWork = def.getShareUnitOfWork() != null && def.getShareUnitOfWork();
+            if (isShareUnitOfWork && child == null) {
+                // only wrap the parent (not the children of the multicast)
+                wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
+            } else {
+                log.trace("{} is part of multicast which have special error handling so no error handler is applied", definition);
+            }
+        } else {
+            // use error handler by default or if configured to do so
+            wrapChannelInErrorHandler(channel, routeContext, inheritErrorHandler);
+        }
+
+        // do post init at the end
+        channel.postInitChannel(definition, routeContext);
+        log.trace("{} wrapped in Channel: {}", definition, channel);
+
+        return channel;
+    }
+
+    /**
+     * Wraps the given channel in error handler (if error handler is inherited)
+     *
+     * @param channel             the channel
+     * @param routeContext        the route context
+     * @param inheritErrorHandler whether to inherit error handler
+     * @throws Exception can be thrown if failed to create error handler builder
+     */
+    private void wrapChannelInErrorHandler(Channel channel, RouteContext routeContext, Boolean inheritErrorHandler) throws Exception {
+        if (inheritErrorHandler == null || inheritErrorHandler) {
+            log.trace("{} is configured to inheritErrorHandler", this);
+            Processor output = channel.getOutput();
+            Processor errorHandler = wrapInErrorHandler(routeContext, output);
+            // set error handler on channel
+            channel.setErrorHandler(errorHandler);
+        } else {
+            log.debug("{} is configured to not inheritErrorHandler.", this);
+        }
+    }
+
+    /**
+     * Wraps the given output in an error handler
+     *
+     * @param routeContext the route context
+     * @param output the output
+     * @return the output wrapped with the error handler
+     * @throws Exception can be thrown if failed to create error handler builder
+     */
+    protected Processor wrapInErrorHandler(RouteContext routeContext, Processor output) throws Exception {
+        ErrorHandlerFactory builder = ((RouteDefinition) routeContext.getRoute()).getErrorHandlerBuilder();
+        // create error handler
+        Processor errorHandler = builder.createErrorHandler(routeContext, output);
+
+        // invoke lifecycles so we can manage this error handler builder
+        for (LifecycleStrategy strategy : routeContext.getCamelContext().getLifecycleStrategies()) {
+            strategy.onErrorHandlerAdd(routeContext, errorHandler, builder);
+        }
+
+        return errorHandler;
+    }
+
+    /**
+     * Adds the given list of interceptors to the channel.
+     *
+     * @param routeContext  the route context
+     * @param channel       the channel to add strategies
+     * @param strategies    list of strategies to add.
+     */
+    protected void addInterceptStrategies(RouteContext routeContext, Channel channel, List<InterceptStrategy> strategies) {
+        for (InterceptStrategy strategy : strategies) {
+            if (!routeContext.isHandleFault() && strategy instanceof HandleFault) {
+                // handle fault is disabled so we should not add it
+                continue;
+            }
+
+            // add strategy
+            channel.addInterceptStrategy(strategy);
+        }
+    }
+
+    /**
+     * Creates a new instance of some kind of composite processor which defaults
+     * to using a {@link Pipeline} but derived classes could change the behaviour
+     */
+    protected Processor createCompositeProcessor(RouteContext routeContext, List<Processor> list) throws Exception {
+        return Pipeline.newInstance(routeContext.getCamelContext(), list);
+    }
+
+    /**
+     * Creates a new instance of the {@link Channel}.
+     */
+    protected ModelChannel createChannel(RouteContext routeContext) throws Exception {
+        return new DefaultChannel();
+    }
+
+    protected Processor createOutputsProcessor(RouteContext routeContext, Collection<ProcessorDefinition<?>> outputs) throws Exception {
+        // We will save list of actions to restore the outputs back to the original state.
+        Runnable propertyPlaceholdersChangeReverter = ProcessorDefinitionHelper.createPropertyPlaceholdersChangeReverter();
+        try {
+            return createOutputsProcessorImpl(routeContext, outputs);
+        } finally {
+            propertyPlaceholdersChangeReverter.run();
+        }
+    }
+
+    protected Processor createOutputsProcessorImpl(RouteContext routeContext, Collection<ProcessorDefinition<?>> outputs) throws Exception {
+        List<Processor> list = new ArrayList<>();
+        for (ProcessorDefinition<?> output : outputs) {
+
+            // allow any custom logic before we create the processor
+            reifier(output).preCreateProcessor();
+
+            // resolve properties before we create the processor
+            ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), output);
+
+            // resolve constant fields (eg Exchange.FILE_NAME)
+            ProcessorDefinitionHelper.resolveKnownConstantFields(output);
+
+            // also resolve properties and constant fields on embedded expressions
+            ProcessorDefinition<?> me = (ProcessorDefinition<?>) output;
+            if (me instanceof ExpressionNode) {
+                ExpressionNode exp = (ExpressionNode) me;
+                ExpressionDefinition expressionDefinition = exp.getExpression();
+                if (expressionDefinition != null) {
+                    // resolve properties before we create the processor
+                    ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
+
+                    // resolve constant fields (eg Exchange.FILE_NAME)
+                    ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
+                }
+            }
+
+            Processor processor = createProcessor(routeContext, output);
+
+            // inject id
+            if (processor instanceof IdAware) {
+                String id = output.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
+                ((IdAware) processor).setId(id);
+            }
+
+            if (output instanceof Channel && processor == null) {
+                continue;
+            }
+
+            Processor channel = wrapChannel(routeContext, processor, output);
+            list.add(channel);
+        }
+
+        // if more than one output wrap than in a composite processor else just keep it as is
+        Processor processor = null;
+        if (!list.isEmpty()) {
+            if (list.size() == 1) {
+                processor = list.get(0);
+            } else {
+                processor = createCompositeProcessor(routeContext, list);
+            }
+        }
+
+        return processor;
+    }
+
+    protected Processor createProcessor(RouteContext routeContext, ProcessorDefinition<?> output) throws Exception {
+        Processor processor = null;
+        // at first use custom factory
+        if (routeContext.getCamelContext().getProcessorFactory() != null) {
+            processor = routeContext.getCamelContext().getProcessorFactory().createProcessor(routeContext, output);
+        }
+        // fallback to default implementation if factory did not create the processor
+        if (processor == null) {
+            processor = reifier(output).createProcessor(routeContext);
+        }
+        return processor;
+    }
+
+    /**
+     * Creates the processor and wraps it in any necessary interceptors and error handlers
+     */
+    protected Processor makeProcessor(RouteContext routeContext) throws Exception {
+        // We will save list of actions to restore the definition back to the original state.
+        Runnable propertyPlaceholdersChangeReverter = ProcessorDefinitionHelper.createPropertyPlaceholdersChangeReverter();
+        try {
+            return makeProcessorImpl(routeContext);
+        } finally {
+            // Lets restore
+            propertyPlaceholdersChangeReverter.run();
+        }
+    }
+
+    private Processor makeProcessorImpl(RouteContext routeContext) throws Exception {
+        Processor processor = null;
+
+        // allow any custom logic before we create the processor
+        preCreateProcessor();
+
+        // resolve properties before we create the processor
+        ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), definition);
+
+        // resolve constant fields (eg Exchange.FILE_NAME)
+        ProcessorDefinitionHelper.resolveKnownConstantFields(definition);
+
+        // also resolve properties and constant fields on embedded expressions
+        ProcessorDefinition<?> me = (ProcessorDefinition<?>) definition;
+        if (me instanceof ExpressionNode) {
+            ExpressionNode exp = (ExpressionNode) me;
+            ExpressionDefinition expressionDefinition = exp.getExpression();
+            if (expressionDefinition != null) {
+                // resolve properties before we create the processor
+                ProcessorDefinitionHelper.resolvePropertyPlaceholders(routeContext.getCamelContext(), expressionDefinition);
+
+                // resolve constant fields (eg Exchange.FILE_NAME)
+                ProcessorDefinitionHelper.resolveKnownConstantFields(expressionDefinition);
+            }
+        }
+
+        // at first use custom factory
+        if (routeContext.getCamelContext().getProcessorFactory() != null) {
+            processor = routeContext.getCamelContext().getProcessorFactory().createProcessor(routeContext, definition);
+        }
+        // fallback to default implementation if factory did not create the processor
+        if (processor == null) {
+            processor = createProcessor(routeContext);
+        }
+
+        // inject id
+        if (processor instanceof IdAware) {
+            String id = definition.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
+            ((IdAware) processor).setId(id);
+        }
+
+        if (processor == null) {
+            // no processor to make
+            return null;
+        }
+        return wrapProcessor(routeContext, processor);
+    }
... 4894 lines suppressed ...