You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by lb...@apache.org on 2017/04/20 08:05:55 UTC

[2/2] camel git commit: CAMEL-11170: Service Call : add a spi for component to provide a custom expression to build the camel uri

CAMEL-11170: Service Call : add a spi for component to provide a custom expression to build the camel uri


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

Branch: refs/heads/master
Commit: 094348372f674f492476e7305527822c65babd94
Parents: 548dcea
Author: lburgazzoli <lb...@gmail.com>
Authored: Wed Apr 19 23:01:19 2017 +0200
Committer: lburgazzoli <lb...@gmail.com>
Committed: Thu Apr 20 10:03:57 2017 +0200

----------------------------------------------------------------------
 .../cloud/DefaultServiceCallExpression.java     | 49 +++++++------
 .../ServiceCallConfigurationDefinition.java     |  2 +-
 .../camel/model/cloud/ServiceCallConstants.java | 30 --------
 .../model/cloud/ServiceCallDefinition.java      | 70 ++++++++++++++-----
 .../cloud/ServiceCallDefinitionConstants.java   | 31 +++++++++
 .../ServiceCallExpressionConfiguration.java     |  6 +-
 .../ServiceCallServiceChooserConfiguration.java |  6 +-
 ...erviceCallServiceDiscoveryConfiguration.java |  6 +-
 .../ServiceCallServiceFilterConfiguration.java  |  6 +-
 ...iceCallServiceLoadBalancerConfiguration.java |  6 +-
 .../cloud/ServiceCallConfigurationTest.java     |  3 +-
 components/camel-consul/pom.xml                 |  5 --
 .../ConsulDefaultServiceCallRouteTest.java      |  1 +
 .../cloud/ConsulRibbonServiceCallRouteTest.java |  1 +
 .../SpringConsulDefaultServiceCallRouteTest.xml |  4 +-
 ...ringConsulExpressionServiceCallRouteTest.xml |  6 +-
 .../SpringConsulRibbonServiceCallRouteTest.xml  |  4 +-
 .../http4/cloud/Http4ServiceExpression.java     | 32 +++++++++
 .../cloud/Http4ServiceExpressionFactory.java    | 28 ++++++++
 .../apache/camel/cloud/http4-service-expression | 17 +++++
 .../camel/cloud/http4s-service-expression       | 17 +++++
 .../jetty/cloud/JettyServiceExpression.java     | 51 ++++++++++++++
 .../cloud/JettyServiceExpressionFactory.java    | 28 ++++++++
 .../apache/camel/cloud/jetty-service-expression | 17 +++++
 .../jetty/cloud/JettyServiceCallRouteTest.java  | 73 ++++++++++++++++++++
 .../http/cloud/Netty4HttpServiceExpression.java | 51 ++++++++++++++
 .../Netty4HttpServiceExpressionFactory.java     | 28 ++++++++
 .../camel/cloud/netty4-http-service-expression  | 17 +++++
 .../cloud/Netty4HttpServiceCallRouteTest.java   | 73 ++++++++++++++++++++
 components/camel-ribbon/pom.xml                 |  5 --
 .../RibbonServiceCallRegistryRouteTest.java     |  5 +-
 .../cloud/RibbonServiceCallRouteDslTest.java    | 13 ++--
 .../cloud/RibbonServiceCallRouteTest.java       |  1 +
 .../cloud/RibbonServiceCallUpdateRouteTest.java |  1 +
 .../SpringBeanRibbonServiceCallRouteTest.xml    |  2 +-
 ...gDslRibbonPropertiesServiceCallRouteTest.xml |  2 +-
 .../SpringDslRibbonServiceCallRouteTest.xml     |  2 +-
 .../CamelCloudConfigurationProperties.java      |  4 +-
 ...rviceCallConfigurationAutoConfiguration.java |  6 +-
 .../CamelCloudServiceCallConfigurationTest.java |  2 +-
 .../boot/cloud/CamelCloudServiceCallTest.java   |  5 +-
 .../cloud/UndertowServiceExpression.java        | 51 ++++++++++++++
 .../cloud/UndertowServiceExpressionFactory.java | 28 ++++++++
 .../camel/cloud/undertow-service-expression     | 17 +++++
 .../cloud/UndertowServiceCallRouteTest.java     | 73 ++++++++++++++++++++
 .../README.adoc                                 |  2 +-
 .../ServiceCallConsumerAutoConfiguration.java   |  4 +-
 .../src/main/resources/application.properties   |  3 +
 .../camel/example/ConsumerApplication.java      |  4 +-
 .../src/main/resources/application.properties   |  3 +
 50 files changed, 771 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/impl/cloud/DefaultServiceCallExpression.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/DefaultServiceCallExpression.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/DefaultServiceCallExpression.java
index f16e59b..ff3bd34 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/cloud/DefaultServiceCallExpression.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/DefaultServiceCallExpression.java
@@ -48,27 +48,7 @@ public class DefaultServiceCallExpression extends ServiceCallExpressionSupport {
         // build basic uri if none provided
         String answer = uri;
         if (answer == null) {
-            if (scheme == null) {
-                // use http/https by default if no scheme or port have been configured
-                if (port == null) {
-                    scheme = "http4";
-                } else if (port == 443) {
-                    scheme = "https4";
-                } else {
-                    scheme = "http4";
-                }
-            }
-            answer = scheme + "://" + host;
-            if (port != null) {
-                answer = answer + ":" + port;
-            }
-            if (contextPath != null) {
-                if (!contextPath.startsWith("/")) {
-                    contextPath = "/" + contextPath;
-                }
-
-                answer += contextPath;
-            }
+            answer = doBuildCamelEndpointUri(host, port, contextPath, scheme);
         } else {
             // we have existing uri, then replace the serviceName with ip:port
             if (answer.contains(name + ".host")) {
@@ -88,4 +68,31 @@ public class DefaultServiceCallExpression extends ServiceCallExpressionSupport {
         LOGGER.debug("Camel endpoint uri: {} for calling service: {} on server {}:{}", answer, name, host, port);
         return answer;
     }
+
+    protected String doBuildCamelEndpointUri(String host, Integer port, String contextPath, String scheme) {
+        if (scheme == null) {
+            // use http/https by default if no scheme or port have been configured
+            if (port == null) {
+                scheme = "http4";
+            } else if (port == 443) {
+                scheme = "http4s";
+            } else {
+                scheme = "http4";
+            }
+        }
+
+        String answer = scheme + "://" + host;
+        if (port != null) {
+            answer = answer + ":" + port;
+        }
+        if (contextPath != null) {
+            if (!contextPath.startsWith("/")) {
+                contextPath = "/" + contextPath;
+            }
+
+            answer += contextPath;
+        }
+
+        return answer;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConfigurationDefinition.java
index a590676..2d8fc96 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConfigurationDefinition.java
@@ -43,7 +43,7 @@ import org.apache.camel.spi.Metadata;
 public class ServiceCallConfigurationDefinition extends IdentifiedType {
     @XmlAttribute
     private String uri;
-    @XmlAttribute @Metadata(defaultValue = ServiceCallConstants.DEFAULT_COMPONENT)
+    @XmlAttribute @Metadata(defaultValue = ServiceCallDefinitionConstants.DEFAULT_COMPONENT)
     private String component;
     @XmlAttribute
     private ExchangePattern pattern;

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConstants.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConstants.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConstants.java
deleted file mode 100644
index 8859f33..0000000
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallConstants.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.camel.model.cloud;
-
-public final class ServiceCallConstants {
-    public static final String DEFAULT_COMPONENT = "http4";
-    public static final String DEFAULT_SERVICE_CALL_CONFIG_ID = "service-call-configuration";
-    public static final String DEFAULT_SERVICE_CALL_EXPRESSION_ID = "service-call-expression";
-    public static final String DEFAULT_SERVICE_DISCOVERY_ID = "service-discovery";
-    public static final String DEFAULT_SERVICE_FILTER_ID = "service-filter";
-    public static final String DEFAULT_SERVICE_CHOOSER_ID = "service-chooser";
-    public static final String DEFAULT_LOAD_BALANCER_ID = "load-balancer";
-
-    private ServiceCallConstants() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinition.java
----------------------------------------------------------------------
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 4da35fc..c9a1547 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,6 +16,7 @@
  */
 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;
@@ -36,6 +37,7 @@ 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;
@@ -49,6 +51,7 @@ import org.apache.camel.impl.cloud.RoundRobinServiceChooser;
 import org.apache.camel.model.NoOutputDefinition;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RouteContext;
+import org.apache.camel.util.CamelContextHelper;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.function.Suppliers;
 
@@ -66,7 +69,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
     private String name;
     @XmlAttribute
     private String uri;
-    @XmlAttribute @Metadata(defaultValue = ServiceCallConstants.DEFAULT_COMPONENT)
+    @XmlAttribute @Metadata(defaultValue = ServiceCallDefinitionConstants.DEFAULT_COMPONENT)
     private String component;
     @XmlAttribute
     private ExchangePattern pattern;
@@ -729,7 +732,6 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
         final ServiceFilter serviceFilter = retrieveServiceFilter(camelContext);
         final ServiceChooser serviceChooser = retrieveServiceChooser(camelContext);
         final ServiceLoadBalancer loadBalancer = retrieveLoadBalancer(camelContext);
-        final Expression expression = retrieveExpression(camelContext);
 
         if (loadBalancer instanceof CamelContextAware) {
             ((CamelContextAware) loadBalancer).setCamelContext(camelContext);
@@ -779,14 +781,17 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
         // Service name is mandatory
         ObjectHelper.notNull(name, "Service name");
 
+        endpointScheme = ObjectHelper.applyIfNotEmpty(endpointScheme, camelContext::resolvePropertyPlaceholders, () -> ServiceCallDefinitionConstants.DEFAULT_COMPONENT);
+        endpointUri = ObjectHelper.applyIfNotEmpty(endpointUri, camelContext::resolvePropertyPlaceholders, () -> null);
+
         return new DefaultServiceCallProcessor(
             camelContext,
             camelContext.resolvePropertyPlaceholders(name),
-            ObjectHelper.applyIfNotEmpty(endpointScheme, camelContext::resolvePropertyPlaceholders, () -> ServiceCallConstants.DEFAULT_COMPONENT),
-            ObjectHelper.applyIfNotEmpty(endpointUri, camelContext::resolvePropertyPlaceholders, () -> null),
+            endpointScheme,
+            endpointUri,
             pattern,
             loadBalancer,
-            expression);
+            retrieveExpression(camelContext, endpointScheme));
     }
 
     // *****************************
@@ -801,7 +806,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Or if it is in the registry
             config = lookup(
                 camelContext,
-                ServiceCallConstants.DEFAULT_SERVICE_CALL_CONFIG_ID,
+                ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_CONFIG_ID,
                 ServiceCallConfigurationDefinition.class);
         }
 
@@ -865,7 +870,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Check if there is a single instance in the registry
             () -> findByType(camelContext, ServiceDiscovery.class),
             // From registry
-            () -> lookup(camelContext, ServiceCallConstants.DEFAULT_SERVICE_DISCOVERY_ID, ServiceDiscovery.class)
+            () -> 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
@@ -921,7 +926,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Check if there is a single instance in the registry
             () -> findByType(camelContext, ServiceFilter.class),
             // From registry
-            () -> lookup(camelContext, ServiceCallConstants.DEFAULT_SERVICE_FILTER_ID, ServiceFilter.class)
+            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_FILTER_ID, ServiceFilter.class)
         ).orElseGet(
             // Default
             () -> new HealthyServiceFilter()
@@ -970,7 +975,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Check if there is a single instance in the registry
             () -> findByType(camelContext, ServiceChooser.class),
             // From registry
-            () -> lookup(camelContext, ServiceCallConstants.DEFAULT_SERVICE_CHOOSER_ID, ServiceChooser.class)
+            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_SERVICE_CHOOSER_ID, ServiceChooser.class)
         ).orElseGet(
             // Default
             () -> new RoundRobinServiceChooser()
@@ -1013,7 +1018,7 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Check if there is a single instance in the registry
             () -> findByType(camelContext, ServiceLoadBalancer.class),
             // From registry
-            () -> lookup(camelContext, ServiceCallConstants.DEFAULT_LOAD_BALANCER_ID, ServiceLoadBalancer.class)
+            () -> lookup(camelContext, ServiceCallDefinitionConstants.DEFAULT_LOAD_BALANCER_ID, ServiceLoadBalancer.class)
         ).orElseGet(
             // Default
             () -> new DefaultServiceLoadBalancer()
@@ -1044,8 +1049,8 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
         return answer;
     }
 
-    private Expression retrieveExpression(CamelContext camelContext) throws Exception {
-        return Suppliers.firstNotNull(
+    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),
@@ -1054,11 +1059,44 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
             // Default configuration
             () -> retrieveExpression(camelContext, this::retrieveDefaultConfig),
             // From registry
-            () -> lookup(camelContext, ServiceCallConstants.DEFAULT_SERVICE_CALL_EXPRESSION_ID, Expression.class)
-        ).orElseGet(
-            // Default
-            () -> new DefaultServiceCallExpression()
+            () -> 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);
+            }
+        }
     }
 
     // ************************************

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinitionConstants.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinitionConstants.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinitionConstants.java
new file mode 100644
index 0000000..63dc519
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallDefinitionConstants.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.model.cloud;
+
+public final class ServiceCallDefinitionConstants {
+    public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
+    public static final String DEFAULT_COMPONENT = "http4";
+    public static final String DEFAULT_SERVICE_CALL_CONFIG_ID = "service-call-configuration";
+    public static final String DEFAULT_SERVICE_CALL_EXPRESSION_ID = "service-call-expression";
+    public static final String DEFAULT_SERVICE_DISCOVERY_ID = "service-discovery";
+    public static final String DEFAULT_SERVICE_FILTER_ID = "service-filter";
+    public static final String DEFAULT_SERVICE_CHOOSER_ID = "service-chooser";
+    public static final String DEFAULT_LOAD_BALANCER_ID = "load-balancer";
+
+    private ServiceCallDefinitionConstants() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallExpressionConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallExpressionConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallExpressionConfiguration.java
index 8a76a69..1565b1b 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallExpressionConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallExpressionConfiguration.java
@@ -49,8 +49,6 @@ import org.apache.camel.util.ObjectHelper;
 @XmlRootElement(name = "serviceExpression")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ServiceCallExpressionConfiguration extends IdentifiedType implements ServiceExpressionFactory {
-    private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
-
     @XmlTransient
     private final ServiceCallDefinition parent;
     @XmlTransient
@@ -230,9 +228,9 @@ public class ServiceCallExpressionConfiguration extends IdentifiedType implement
                 Class<?> type;
                 try {
                     // Then use Service factory.
-                    type = camelContext.getFactoryFinder(RESOURCE_PATH).findClass(factoryKey);
+                    type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(factoryKey);
                 } catch (Exception e) {
-                    throw new NoFactoryAvailableException(RESOURCE_PATH + factoryKey, e);
+                    throw new NoFactoryAvailableException(ServiceCallDefinitionConstants.RESOURCE_PATH + factoryKey, e);
                 }
 
                 if (type != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceChooserConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceChooserConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceChooserConfiguration.java
index b6c6e7e..0228f72 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceChooserConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceChooserConfiguration.java
@@ -44,8 +44,6 @@ import org.apache.camel.util.ObjectHelper;
 @XmlRootElement(name = "serviceChooserConfiguration")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ServiceCallServiceChooserConfiguration extends IdentifiedType implements ServiceChooserFactory {
-    private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
-
     @XmlTransient
     private final ServiceCallDefinition parent;
     @XmlTransient
@@ -146,9 +144,9 @@ public class ServiceCallServiceChooserConfiguration extends IdentifiedType imple
             Class<?> type;
             try {
                 // Then use Service factory.
-                type = camelContext.getFactoryFinder(RESOURCE_PATH).findClass(factoryKey);
+                type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(factoryKey);
             } catch (Exception e) {
-                throw new NoFactoryAvailableException(RESOURCE_PATH + factoryKey, e);
+                throw new NoFactoryAvailableException(ServiceCallDefinitionConstants.RESOURCE_PATH + factoryKey, e);
             }
 
             if (type != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceDiscoveryConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceDiscoveryConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceDiscoveryConfiguration.java
index baf1773..49e6c84 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceDiscoveryConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceDiscoveryConfiguration.java
@@ -45,8 +45,6 @@ import org.apache.camel.util.ObjectHelper;
 @XmlRootElement(name = "serviceDiscoveryConfiguration")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType implements ServiceDiscoveryFactory {
-    private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
-
     @XmlTransient
     private final Optional<ServiceCallDefinition> parent;
     @XmlTransient
@@ -153,9 +151,9 @@ public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType imp
             Class<?> type;
             try {
                 // Then use Service factory.
-                type = camelContext.getFactoryFinder(RESOURCE_PATH).findClass(factoryKey);
+                type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(factoryKey);
             } catch (Exception e) {
-                throw new NoFactoryAvailableException(RESOURCE_PATH + factoryKey, e);
+                throw new NoFactoryAvailableException(ServiceCallDefinitionConstants.RESOURCE_PATH + factoryKey, e);
             }
 
             if (type != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceFilterConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceFilterConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceFilterConfiguration.java
index b5dc423..afb2e94 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceFilterConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceFilterConfiguration.java
@@ -44,8 +44,6 @@ import org.apache.camel.util.ObjectHelper;
 @XmlRootElement(name = "serviceFilterConfiguration")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ServiceCallServiceFilterConfiguration extends IdentifiedType implements ServiceFilterFactory {
-    private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
-
     @XmlTransient
     private final ServiceCallDefinition parent;
     @XmlTransient
@@ -146,9 +144,9 @@ public class ServiceCallServiceFilterConfiguration extends IdentifiedType implem
             Class<?> type;
             try {
                 // Then use Service factory.
-                type = camelContext.getFactoryFinder(RESOURCE_PATH).findClass(factoryKey);
+                type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(factoryKey);
             } catch (Exception e) {
-                throw new NoFactoryAvailableException(RESOURCE_PATH + factoryKey, e);
+                throw new NoFactoryAvailableException(ServiceCallDefinitionConstants.RESOURCE_PATH + factoryKey, e);
             }
 
             if (type != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceLoadBalancerConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceLoadBalancerConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceLoadBalancerConfiguration.java
index 5c25f10..a1057a4 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceLoadBalancerConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallServiceLoadBalancerConfiguration.java
@@ -44,8 +44,6 @@ import org.apache.camel.util.ObjectHelper;
 @XmlRootElement(name = "loadBalancerConfiguration")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ServiceCallServiceLoadBalancerConfiguration extends IdentifiedType implements ServiceLoadBalancerFactory {
-    private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
-
     @XmlTransient
     private final ServiceCallDefinition parent;
     @XmlTransient
@@ -146,9 +144,9 @@ public class ServiceCallServiceLoadBalancerConfiguration extends IdentifiedType
             Class<?> type;
             try {
                 // Then use Service factory.
-                type = camelContext.getFactoryFinder(RESOURCE_PATH).findClass(factoryKey);
+                type = camelContext.getFactoryFinder(ServiceCallDefinitionConstants.RESOURCE_PATH).findClass(factoryKey);
             } catch (Exception e) {
-                throw new NoFactoryAvailableException(RESOURCE_PATH + factoryKey, e);
+                throw new NoFactoryAvailableException(ServiceCallDefinitionConstants.RESOURCE_PATH + factoryKey, e);
             }
 
             if (type != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/camel-core/src/test/java/org/apache/camel/impl/cloud/ServiceCallConfigurationTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/cloud/ServiceCallConfigurationTest.java b/camel-core/src/test/java/org/apache/camel/impl/cloud/ServiceCallConfigurationTest.java
index 155bcc0..67c0105 100644
--- a/camel-core/src/test/java/org/apache/camel/impl/cloud/ServiceCallConfigurationTest.java
+++ b/camel-core/src/test/java/org/apache/camel/impl/cloud/ServiceCallConfigurationTest.java
@@ -25,6 +25,7 @@ import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.impl.SimpleRegistry;
 import org.apache.camel.model.cloud.ServiceCallConfigurationDefinition;
+import org.apache.camel.model.cloud.ServiceCallDefinitionConstants;
 import org.apache.camel.model.cloud.ServiceCallExpressionConfiguration;
 import org.apache.camel.model.language.SimpleExpression;
 import org.junit.Assert;
@@ -91,7 +92,7 @@ public class ServiceCallConfigurationTest {
         conf.serviceFilter(sf);
 
         SimpleRegistry reg = new SimpleRegistry();
-        reg.put(org.apache.camel.model.cloud.ServiceCallConstants.DEFAULT_SERVICE_CALL_CONFIG_ID, conf);
+        reg.put(ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_CONFIG_ID, conf);
 
         CamelContext context = new DefaultCamelContext(reg);
         context.addRoutes(new RouteBuilder() {

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-consul/pom.xml b/components/camel-consul/pom.xml
index dd1a4d3..63ae87c 100644
--- a/components/camel-consul/pom.xml
+++ b/components/camel-consul/pom.xml
@@ -64,11 +64,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.camel</groupId>
-      <artifactId>camel-http4</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.camel</groupId>
       <artifactId>camel-jetty</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java
index fcb4479..00586db 100644
--- a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java
@@ -96,6 +96,7 @@ public class ConsulDefaultServiceCallRouteTest extends ConsulTestSupport {
                 from("direct:start")
                     .serviceCall()
                         .name(SERVICE_NAME)
+                        .component("jetty")
                         .defaultLoadBalancer()
                         .consulServiceDiscovery()
                         .endParent()

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java
index b154835..542e6cb 100644
--- a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java
@@ -97,6 +97,7 @@ public class ConsulRibbonServiceCallRouteTest extends ConsulTestSupport {
                 from("direct:start")
                     .serviceCall()
                         .name(SERVICE_NAME)
+                        .component("jetty")
                         .consulServiceDiscovery()
                         .endParent()
                     .to("log:org.apache.camel.component.consul.processor.service?level=INFO&showAll=true&multiline=true")

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulDefaultServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulDefaultServiceCallRouteTest.xml b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulDefaultServiceCallRouteTest.xml
index 5e8731e..08c7814 100644
--- a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulDefaultServiceCallRouteTest.xml
+++ b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulDefaultServiceCallRouteTest.xml
@@ -29,7 +29,7 @@
     <!-- ServiceCall                           -->
     <!-- ************************************* -->
 
-    <defaultServiceCallConfiguration id="default">
+    <defaultServiceCallConfiguration id="default" component="jetty">
       <!-- service discovery -->
       <consulServiceDiscovery url="http://localhost:8500"/>
 
@@ -40,7 +40,7 @@
       </blacklistServiceFilter>
     </defaultServiceCallConfiguration>
 
-    <serviceCallConfiguration id="service-2">
+    <serviceCallConfiguration id="service-2" component="jetty">
       <!-- service filter -->
       <blacklistServiceFilter>
         <servers>http-service-2@127.0.0.1:9022</servers>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulExpressionServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulExpressionServiceCallRouteTest.xml b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulExpressionServiceCallRouteTest.xml
index a31e545..f593423 100644
--- a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulExpressionServiceCallRouteTest.xml
+++ b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulExpressionServiceCallRouteTest.xml
@@ -29,7 +29,7 @@
     <!-- ServiceCall                           -->
     <!-- ************************************* -->
 
-    <defaultServiceCallConfiguration id="default">
+    <defaultServiceCallConfiguration id="default" component="jetty">
       <!-- service discovery -->
       <consulServiceDiscovery url="http://localhost:8500"/>
 
@@ -40,13 +40,13 @@
       </blacklistServiceFilter>
     </defaultServiceCallConfiguration>
 
-    <serviceCallConfiguration id="service-2">
+    <serviceCallConfiguration id="service-2" component="jetty">
       <!-- service filter -->
       <blacklistServiceFilter>
         <servers>http-service-2@127.0.0.1:9022</servers>
       </blacklistServiceFilter>
       <expression>
-        <simple>http4:${header.CamelServiceCallServiceHost}:${header.CamelServiceCallServicePort}/hello</simple>
+        <simple>jetty:http://${header.CamelServiceCallServiceHost}:${header.CamelServiceCallServicePort}/hello</simple>
       </expression>
     </serviceCallConfiguration>
 

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulRibbonServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulRibbonServiceCallRouteTest.xml b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulRibbonServiceCallRouteTest.xml
index 5af397c..2ca0c81 100644
--- a/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulRibbonServiceCallRouteTest.xml
+++ b/components/camel-consul/src/test/resources/org/apache/camel/component/consul/cloud/SpringConsulRibbonServiceCallRouteTest.xml
@@ -29,7 +29,7 @@
     <!-- ServiceCall                           -->
     <!-- ************************************* -->
 
-    <defaultServiceCallConfiguration id="default">
+    <defaultServiceCallConfiguration id="default" component="jetty">
       <!-- service discovery -->
       <consulServiceDiscovery url="http://localhost:8500"/>
 
@@ -40,7 +40,7 @@
       </blacklistServiceFilter>
     </defaultServiceCallConfiguration>
 
-    <serviceCallConfiguration id="service-2">
+    <serviceCallConfiguration id="service-2" component="jetty">
       <!-- service filter -->
       <blacklistServiceFilter>
         <servers>http-service-2@127.0.0.1:9022</servers>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpression.java
----------------------------------------------------------------------
diff --git a/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpression.java b/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpression.java
new file mode 100644
index 0000000..2276538
--- /dev/null
+++ b/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpression.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.http4.cloud;
+
+import org.apache.camel.impl.cloud.DefaultServiceCallExpression;
+
+/**
+ * The default is based on camel-http4, this class is added to allow further
+ * customizations.
+ */
+public final class Http4ServiceExpression extends DefaultServiceCallExpression {
+    public Http4ServiceExpression() {
+    }
+
+    public Http4ServiceExpression(String hostHeader, String portHeader) {
+        super(hostHeader, portHeader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpressionFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpressionFactory.java b/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpressionFactory.java
new file mode 100644
index 0000000..346242b
--- /dev/null
+++ b/components/camel-http4/src/main/java/org/apache/camel/component/http4/cloud/Http4ServiceExpressionFactory.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.http4.cloud;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Expression;
+import org.apache.camel.cloud.ServiceExpressionFactory;
+
+public class Http4ServiceExpressionFactory implements ServiceExpressionFactory {
+    @Override
+    public Expression newInstance(CamelContext camelContext) throws Exception {
+        return new Http4ServiceExpression();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4-service-expression
----------------------------------------------------------------------
diff --git a/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4-service-expression b/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4-service-expression
new file mode 100644
index 0000000..60bc936
--- /dev/null
+++ b/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4-service-expression
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+class=org.apache.camel.component.http4.cloud.Http4ServiceExpressionFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4s-service-expression
----------------------------------------------------------------------
diff --git a/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4s-service-expression b/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4s-service-expression
new file mode 100644
index 0000000..60bc936
--- /dev/null
+++ b/components/camel-http4/src/main/resources/META-INF/services/org/apache/camel/cloud/http4s-service-expression
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+class=org.apache.camel.component.http4.cloud.Http4ServiceExpressionFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpression.java
----------------------------------------------------------------------
diff --git a/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpression.java b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpression.java
new file mode 100644
index 0000000..48cb30a
--- /dev/null
+++ b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpression.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.jetty.cloud;
+
+import org.apache.camel.impl.cloud.DefaultServiceCallExpression;
+import org.apache.camel.util.ObjectHelper;
+
+public final class JettyServiceExpression extends DefaultServiceCallExpression {
+    public JettyServiceExpression() {
+    }
+
+    public JettyServiceExpression(String hostHeader, String portHeader) {
+        super(hostHeader, portHeader);
+    }
+
+    @Override
+    protected String doBuildCamelEndpointUri(String host, Integer port, String contextPath, String scheme) {
+        if (!ObjectHelper.equal(scheme, "jetty")) {
+            return super.doBuildCamelEndpointUri(host, port, contextPath, scheme);
+        }
+
+        String answer = scheme + ":http://" + host;
+        if (port != null) {
+            answer = answer + ":" + port;
+        }
+
+        if (contextPath != null) {
+            if (!contextPath.startsWith("/")) {
+                contextPath = "/" + contextPath;
+            }
+
+            answer += contextPath;
+        }
+
+        return answer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpressionFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpressionFactory.java b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpressionFactory.java
new file mode 100644
index 0000000..ffe8df4
--- /dev/null
+++ b/components/camel-jetty-common/src/main/java/org/apache/camel/component/jetty/cloud/JettyServiceExpressionFactory.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.jetty.cloud;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Expression;
+import org.apache.camel.cloud.ServiceExpressionFactory;
+
+public class JettyServiceExpressionFactory implements ServiceExpressionFactory {
+    @Override
+    public Expression newInstance(CamelContext camelContext) throws Exception {
+        return new JettyServiceExpression();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-jetty-common/src/main/resources/META-INF/services/org/apache/camel/cloud/jetty-service-expression
----------------------------------------------------------------------
diff --git a/components/camel-jetty-common/src/main/resources/META-INF/services/org/apache/camel/cloud/jetty-service-expression b/components/camel-jetty-common/src/main/resources/META-INF/services/org/apache/camel/cloud/jetty-service-expression
new file mode 100644
index 0000000..8932495
--- /dev/null
+++ b/components/camel-jetty-common/src/main/resources/META-INF/services/org/apache/camel/cloud/jetty-service-expression
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+class=org.apache.camel.component.jetty.cloud.JettyServiceExpressionFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-jetty9/src/test/java/org/apache/camel/component/jetty/cloud/JettyServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jetty9/src/test/java/org/apache/camel/component/jetty/cloud/JettyServiceCallRouteTest.java b/components/camel-jetty9/src/test/java/org/apache/camel/component/jetty/cloud/JettyServiceCallRouteTest.java
new file mode 100644
index 0000000..de57a14
--- /dev/null
+++ b/components/camel-jetty9/src/test/java/org/apache/camel/component/jetty/cloud/JettyServiceCallRouteTest.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.jetty.cloud;
+
+import org.apache.camel.ResolveEndpointFailedException;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.RuntimeCamelException;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JettyServiceCallRouteTest extends CamelTestSupport {
+
+    @Test
+    public void testCustomSchema() throws Exception {
+        Assert.assertEquals("8081", template.requestBody("direct:custom", "hello", String.class));
+        Assert.assertEquals("8082", template.requestBody("direct:custom", "hello", String.class));
+    }
+
+    @Test
+    public void testDefaultSchema() throws Exception {
+        try {
+            Assert.assertEquals("8081", template.requestBody("direct:default", "hello", String.class));
+        } catch (RuntimeCamelException e) {
+            assertTrue(e.getCause() instanceof ResolveEndpointFailedException);
+        }
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:custom")
+                    .serviceCall()
+                        .name("myService")
+                        .component("jetty")
+                        .staticServiceDiscovery()
+                            .server("myService@localhost:8081")
+                            .server("myService@localhost:8082")
+                        .endParent();
+
+                from("direct:default")
+                    .serviceCall()
+                        .name("myService")
+                        .staticServiceDiscovery()
+                            .server("myService@localhost:8081")
+                            .server("myService@localhost:8082")
+                        .endParent();
+
+                from("jetty:http://localhost:8081")
+                    .transform().constant("8081");
+                from("jetty:http://localhost:8082")
+                    .transform().constant("8082");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpression.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpression.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpression.java
new file mode 100644
index 0000000..a0fbfde
--- /dev/null
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpression.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.netty4.http.cloud;
+
+import org.apache.camel.impl.cloud.DefaultServiceCallExpression;
+import org.apache.camel.util.ObjectHelper;
+
+public final class Netty4HttpServiceExpression extends DefaultServiceCallExpression {
+    public Netty4HttpServiceExpression() {
+    }
+
+    public Netty4HttpServiceExpression(String hostHeader, String portHeader) {
+        super(hostHeader, portHeader);
+    }
+
+    @Override
+    protected String doBuildCamelEndpointUri(String host, Integer port, String contextPath, String scheme) {
+        if (!ObjectHelper.equal(scheme, "netty4-http")) {
+            return super.doBuildCamelEndpointUri(host, port, contextPath, scheme);
+        }
+
+        String answer = scheme + ":http://" + host;
+        if (port != null) {
+            answer = answer + ":" + port;
+        }
+
+        if (contextPath != null) {
+            if (!contextPath.startsWith("/")) {
+                contextPath = "/" + contextPath;
+            }
+
+            answer += contextPath;
+        }
+
+        return answer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpressionFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpressionFactory.java b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpressionFactory.java
new file mode 100644
index 0000000..94cfc94
--- /dev/null
+++ b/components/camel-netty4-http/src/main/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceExpressionFactory.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.netty4.http.cloud;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Expression;
+import org.apache.camel.cloud.ServiceExpressionFactory;
+
+public class Netty4HttpServiceExpressionFactory implements ServiceExpressionFactory {
+    @Override
+    public Expression newInstance(CamelContext camelContext) throws Exception {
+        return new Netty4HttpServiceExpression();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-netty4-http/src/main/resources/META-INF/services/org/apache/camel/cloud/netty4-http-service-expression
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/main/resources/META-INF/services/org/apache/camel/cloud/netty4-http-service-expression b/components/camel-netty4-http/src/main/resources/META-INF/services/org/apache/camel/cloud/netty4-http-service-expression
new file mode 100644
index 0000000..976e275
--- /dev/null
+++ b/components/camel-netty4-http/src/main/resources/META-INF/services/org/apache/camel/cloud/netty4-http-service-expression
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+class=org.apache.camel.component.netty4.http.cloud.Netty4ServiceExpressionFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceCallRouteTest.java b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceCallRouteTest.java
new file mode 100644
index 0000000..0a610e8
--- /dev/null
+++ b/components/camel-netty4-http/src/test/java/org/apache/camel/component/netty4/http/cloud/Netty4HttpServiceCallRouteTest.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.netty4.http.cloud;
+
+import org.apache.camel.ResolveEndpointFailedException;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.RuntimeCamelException;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class Netty4HttpServiceCallRouteTest extends CamelTestSupport {
+
+    @Test
+    public void testCustomCall() throws Exception {
+        Assert.assertEquals("8081", template.requestBody("direct:custom", "hello", String.class));
+        Assert.assertEquals("8082", template.requestBody("direct:custom", "hello", String.class));
+    }
+
+    @Test
+    public void testDefaultSchema() throws Exception {
+        try {
+            Assert.assertEquals("8081", template.requestBody("direct:default", "hello", String.class));
+        } catch (RuntimeCamelException e) {
+            assertTrue(e.getCause() instanceof ResolveEndpointFailedException);
+        }
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:custom")
+                    .serviceCall()
+                        .name("myService")
+                        .component("netty4-http")
+                        .staticServiceDiscovery()
+                            .server("myService@localhost:8081")
+                            .server("myService@localhost:8082")
+                        .endParent();
+
+                from("direct:default")
+                    .serviceCall()
+                        .name("myService")
+                        .staticServiceDiscovery()
+                            .server("myService@localhost:8081")
+                            .server("myService@localhost:8082")
+                        .endParent();
+
+                from("netty4-http:http://localhost:8081")
+                    .transform().constant("8081");
+                from("netty4-http:http://localhost:8082")
+                    .transform().constant("8082");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/pom.xml b/components/camel-ribbon/pom.xml
index 7ad8c5e..c4201ff 100644
--- a/components/camel-ribbon/pom.xml
+++ b/components/camel-ribbon/pom.xml
@@ -65,11 +65,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.camel</groupId>
-      <artifactId>camel-http4</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.camel</groupId>
       <artifactId>camel-jetty</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRegistryRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRegistryRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRegistryRouteTest.java
index 5be0344..635c530 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRegistryRouteTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRegistryRouteTest.java
@@ -46,7 +46,10 @@ public class RibbonServiceCallRegistryRouteTest extends RibbonServiceCallRouteTe
                 context.setServiceCallConfiguration(config);
 
                 from("direct:start")
-                    .serviceCall("myService")
+                    .serviceCall()
+                        .name("myService")
+                        .component("jetty")
+                        .end()
                     .to("mock:result");
                 from("jetty:http://localhost:9090")
                     .to("mock:9090")

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java
index 28a2897..f59379c 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java
@@ -43,12 +43,13 @@ public class RibbonServiceCallRouteDslTest extends CamelTestSupport {
             public void configure() throws Exception {
                 from("direct:start")
                     .serviceCall()
-                    .name("myService")
-                    .ribbonLoadBalancer()
-                    .staticServiceDiscovery()
-                        .server("localhost:9090")
-                        .server("localhost:9091")
-                    .endParent()
+                        .name("myService")
+                        .component("jetty")
+                        .ribbonLoadBalancer()
+                        .staticServiceDiscovery()
+                            .server("localhost:9090")
+                            .server("localhost:9091")
+                        .endParent()
                     .to("mock:result");
                 from("jetty:http://localhost:9090")
                     .to("mock:9090")

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteTest.java
index 1b87c7a..c7990de 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteTest.java
@@ -55,6 +55,7 @@ public class RibbonServiceCallRouteTest extends CamelTestSupport {
                 from("direct:start")
                     .serviceCall()
                         .name("myService")
+                        .component("jetty")
                         .loadBalancer(loadBalancer)
                         .serviceDiscovery(servers)
                         .end()

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallUpdateRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallUpdateRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallUpdateRouteTest.java
index 38efa72..867931f 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallUpdateRouteTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallUpdateRouteTest.java
@@ -80,6 +80,7 @@ public class RibbonServiceCallUpdateRouteTest extends CamelTestSupport {
                 from("direct:start")
                     .serviceCall()
                         .name("myService")
+                        .component("jetty")
                         .loadBalancer(loadBalancer)
                         .serviceDiscovery(servers)
                         .end()

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringBeanRibbonServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringBeanRibbonServiceCallRouteTest.xml b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringBeanRibbonServiceCallRouteTest.xml
index 22ad55c..62e2947 100644
--- a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringBeanRibbonServiceCallRouteTest.xml
+++ b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringBeanRibbonServiceCallRouteTest.xml
@@ -41,7 +41,7 @@
 
     <route id="scall">
       <from uri="direct:start"/>
-      <serviceCall name="myService"/>
+      <serviceCall name="myService" component="jetty"/>
       <to uri="mock:result"/>
     </route>
 

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonPropertiesServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonPropertiesServiceCallRouteTest.xml b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonPropertiesServiceCallRouteTest.xml
index 3c92e91..ee7886a 100644
--- a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonPropertiesServiceCallRouteTest.xml
+++ b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonPropertiesServiceCallRouteTest.xml
@@ -26,7 +26,7 @@
   <camelContext xmlns="http://camel.apache.org/schema/spring">
     <route id="scall">
       <from uri="direct:start"/>
-      <serviceCall name="myService">
+      <serviceCall name="myService" component="jetty">
         <!-- enable ribbon load balancer -->
         <ribbonLoadBalancer clientName="myClient">
           <properties key="listOfServers" value="localhost:9090,localhost:9091"/>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonServiceCallRouteTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonServiceCallRouteTest.xml b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonServiceCallRouteTest.xml
index 8a40924..2154111 100644
--- a/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonServiceCallRouteTest.xml
+++ b/components/camel-ribbon/src/test/resources/org/apache/camel/component/ribbon/cloud/SpringDslRibbonServiceCallRouteTest.xml
@@ -26,7 +26,7 @@
   <camelContext xmlns="http://camel.apache.org/schema/spring">
     <route id="scall">
       <from uri="direct:start"/>
-      <serviceCall name="myService">
+      <serviceCall name="myService" component="jetty">
         <!-- static list of servers -->
         <staticServiceDiscovery>
           <servers>localhost:9090,localhost:9091</servers>

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudConfigurationProperties.java
----------------------------------------------------------------------
diff --git a/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudConfigurationProperties.java b/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudConfigurationProperties.java
index 31ebf09..7fbd65d 100644
--- a/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudConfigurationProperties.java
+++ b/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudConfigurationProperties.java
@@ -22,7 +22,7 @@ import java.util.Map;
 
 import org.apache.camel.Expression;
 import org.apache.camel.cloud.ServiceLoadBalancer;
-import org.apache.camel.model.cloud.ServiceCallConstants;
+import org.apache.camel.model.cloud.ServiceCallDefinitionConstants;
 import org.springframework.boot.context.properties.ConfigurationProperties;
 
 @ConfigurationProperties(prefix = "camel.cloud")
@@ -76,7 +76,7 @@ public class CamelCloudConfigurationProperties {
         /**
          * The component to use.
          */
-        private String component = ServiceCallConstants.DEFAULT_COMPONENT;
+        private String component = ServiceCallDefinitionConstants.DEFAULT_COMPONENT;
 
         /**
          * A reference to the {@link org.apache.camel.cloud.ServiceDiscovery} to use.

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationAutoConfiguration.java
----------------------------------------------------------------------
diff --git a/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationAutoConfiguration.java b/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationAutoConfiguration.java
index 84bdcab..79edbf4 100644
--- a/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationAutoConfiguration.java
+++ b/components/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationAutoConfiguration.java
@@ -19,7 +19,7 @@ package org.apache.camel.spring.boot.cloud;
 import org.apache.camel.CamelContext;
 import org.apache.camel.Expression;
 import org.apache.camel.model.cloud.ServiceCallConfigurationDefinition;
-import org.apache.camel.model.cloud.ServiceCallConstants;
+import org.apache.camel.model.cloud.ServiceCallDefinitionConstants;
 import org.apache.camel.model.language.RefExpression;
 import org.apache.camel.model.language.SimpleExpression;
 import org.apache.camel.spi.Language;
@@ -45,8 +45,8 @@ public class CamelCloudServiceCallConfigurationAutoConfiguration {
     private CamelCloudConfigurationProperties configurationProperties;
 
     @Lazy
-    @Bean(name = ServiceCallConstants.DEFAULT_SERVICE_CALL_CONFIG_ID)
-    @ConditionalOnMissingBean(name = ServiceCallConstants.DEFAULT_SERVICE_CALL_CONFIG_ID)
+    @Bean(name = ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_CONFIG_ID)
+    @ConditionalOnMissingBean(name = ServiceCallDefinitionConstants.DEFAULT_SERVICE_CALL_CONFIG_ID)
     public ServiceCallConfigurationDefinition serviceCallConfiguration() throws Exception {
         ServiceCallConfigurationDefinition definition = new ServiceCallConfigurationDefinition();
         ObjectHelper.ifNotEmpty(configurationProperties.getServiceCall().getComponent(), definition::setComponent);

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationTest.java b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationTest.java
index 3f3415a..4226323 100644
--- a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationTest.java
+++ b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallConfigurationTest.java
@@ -17,10 +17,10 @@
 
 package org.apache.camel.spring.boot.cloud;
 
-import org.apache.camel.cloud.ServiceLoadBalancer;
 import org.apache.camel.cloud.ServiceChooser;
 import org.apache.camel.cloud.ServiceDiscovery;
 import org.apache.camel.cloud.ServiceFilter;
+import org.apache.camel.cloud.ServiceLoadBalancer;
 import org.apache.camel.spring.boot.CamelAutoConfiguration;
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallTest.java
----------------------------------------------------------------------
diff --git a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallTest.java b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallTest.java
index 1409244..1c1dd48 100644
--- a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallTest.java
+++ b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/cloud/CamelCloudServiceCallTest.java
@@ -40,6 +40,7 @@ import org.springframework.test.context.junit4.SpringRunner;
     },
     properties = {
         "camel.cloud.load-balancer.enabled=false",
+        "camel.cloud.service-call.component=jetty",
         "camel.cloud.service-discovery.services[custom-svc-list]=localhost:9090,localhost:9091,localhost:9092",
         "camel.cloud.service-filter.blacklist[custom-svc-list]=localhost:9091",
         "ribbon.enabled=false",
@@ -68,9 +69,7 @@ public class CamelCloudServiceCallTest {
                 @Override
                 public void configure() throws Exception {
                     from("direct:start")
-                        .serviceCall()
-                            .name("custom-svc-list")
-                            .uri("jetty:http://custom-svc-list/hello");
+                        .serviceCall("custom-svc-list/hello");
 
                     from("jetty:http://localhost:9090/hello")
                         .transform()

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpression.java
----------------------------------------------------------------------
diff --git a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpression.java b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpression.java
new file mode 100644
index 0000000..235e16e
--- /dev/null
+++ b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpression.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.undertow.cloud;
+
+import org.apache.camel.impl.cloud.DefaultServiceCallExpression;
+import org.apache.camel.util.ObjectHelper;
+
+public final class UndertowServiceExpression extends DefaultServiceCallExpression {
+    public UndertowServiceExpression() {
+    }
+
+    public UndertowServiceExpression(String hostHeader, String portHeader) {
+        super(hostHeader, portHeader);
+    }
+
+    @Override
+    protected String doBuildCamelEndpointUri(String host, Integer port, String contextPath, String scheme) {
+        if (!ObjectHelper.equal(scheme, "undertow")) {
+            return super.doBuildCamelEndpointUri(host, port, contextPath, scheme);
+        }
+
+        String answer = scheme + ":http://" + host;
+        if (port != null) {
+            answer = answer + ":" + port;
+        }
+
+        if (contextPath != null) {
+            if (!contextPath.startsWith("/")) {
+                contextPath = "/" + contextPath;
+            }
+
+            answer += contextPath;
+        }
+
+        return answer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpressionFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpressionFactory.java b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpressionFactory.java
new file mode 100644
index 0000000..b42f5b2
--- /dev/null
+++ b/components/camel-undertow/src/main/java/org/apache/camel/component/undertow/cloud/UndertowServiceExpressionFactory.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.undertow.cloud;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Expression;
+import org.apache.camel.cloud.ServiceExpressionFactory;
+
+public class UndertowServiceExpressionFactory implements ServiceExpressionFactory {
+    @Override
+    public Expression newInstance(CamelContext camelContext) throws Exception {
+        return new UndertowServiceExpression();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/09434837/components/camel-undertow/src/main/resources/META-INF/services/org/apache/camel/cloud/undertow-service-expression
----------------------------------------------------------------------
diff --git a/components/camel-undertow/src/main/resources/META-INF/services/org/apache/camel/cloud/undertow-service-expression b/components/camel-undertow/src/main/resources/META-INF/services/org/apache/camel/cloud/undertow-service-expression
new file mode 100644
index 0000000..2b8f247
--- /dev/null
+++ b/components/camel-undertow/src/main/resources/META-INF/services/org/apache/camel/cloud/undertow-service-expression
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+class=org.apache.camel.component.undertow.cloud.UndertowServiceExpressionFactory
\ No newline at end of file