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/01/16 16:08:38 UTC

[1/2] camel git commit: CAMEL-10710: ServiceCall EIP : improve Java DSL

Repository: camel
Updated Branches:
  refs/heads/master fdb508a84 -> 80f746dfd


http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/cloud/EtcdServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/cloud/EtcdServiceCallRouteTest.java b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/cloud/EtcdServiceCallRouteTest.java
index 766eeca..85b1d64 100644
--- a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/cloud/EtcdServiceCallRouteTest.java
+++ b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/cloud/EtcdServiceCallRouteTest.java
@@ -27,8 +27,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import mousio.etcd4j.EtcdClient;
 import org.apache.camel.RoutesBuilder;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.cloud.ServiceDiscovery;
-import org.apache.camel.component.etcd.EtcdConfiguration;
 import org.apache.camel.component.etcd.EtcdHelper;
 import org.apache.camel.component.etcd.EtcdTestSupport;
 import org.junit.Test;
@@ -96,15 +94,12 @@ public class EtcdServiceCallRouteTest extends EtcdTestSupport {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                EtcdConfiguration configuration = new EtcdConfiguration(null);
-                ServiceDiscovery discovery = new EtcdOnDemandServiceDiscovery(configuration);
-
                 from("direct:start")
                     .serviceCall()
                         .name(SERVICE_NAME)
-                        .component("http")
-                        .serviceDiscovery(discovery)
-                        .end()
+                        .etcdServiceDiscovery()
+                            .type("on-demand")
+                        .endParent()
                     .to("log:org.apache.camel.component.etcd.processor.service?level=INFO&showAll=true&multiline=true")
                     .to("mock:result");
 

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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
new file mode 100644
index 0000000..c2f8b67
--- /dev/null
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/cloud/RibbonServiceCallRouteDslTest.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.ribbon.cloud;
+
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class RibbonServiceCallRouteDslTest extends CamelTestSupport {
+    @Test
+    public void testServiceCall() throws Exception {
+        getMockEndpoint("mock:9090").expectedMessageCount(1);
+        getMockEndpoint("mock:9091").expectedMessageCount(1);
+        getMockEndpoint("mock:result").expectedMessageCount(2);
+
+        String out = template.requestBody("direct:start", null, String.class);
+        String out2 = template.requestBody("direct:start", null, String.class);
+        assertEquals("9091", out);
+        assertEquals("9090", out2);
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .serviceCall()
+                    .name("myService")
+                    .ribbonLoadBalancer()
+                        .end()
+                    .staticServiceDiscovery()
+                        .server("localhost:9090")
+                        .server("localhost:9091")
+                        .end()
+                    .end()
+                    .to("mock:result");
+                from("jetty:http://localhost:9090")
+                    .to("mock:9090")
+                    .transform().constant("9090");
+                from("jetty:http://localhost:9091")
+                    .to("mock:9091")
+                    .transform().constant("9091");
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/components/camel-spring/src/test/java/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-spring/src/test/java/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.java b/components/camel-spring/src/test/java/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.java
index fe3371e..97cac01 100644
--- a/components/camel-spring/src/test/java/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.java
+++ b/components/camel-spring/src/test/java/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.spring.cloud;
 
+import org.apache.camel.model.cloud.MultiServiceCallServiceDiscoveryConfiguration;
 import org.apache.camel.model.cloud.ServiceCallConfigurationDefinition;
 import org.apache.camel.model.cloud.StaticServiceCallServiceDiscoveryConfiguration;
 import org.apache.camel.spring.SpringCamelContext;
@@ -24,6 +25,7 @@ import org.springframework.context.support.ClassPathXmlApplicationContext;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class ServiceCallConfigurationTest {
     @Test
@@ -42,10 +44,18 @@ public class ServiceCallConfigurationTest {
         assertNotNull("No ServiceCallConfiguration (2)", conf2);
         assertNotNull("No ServiceDiscoveryConfiguration (2)", conf2.getServiceDiscoveryConfiguration());
 
-        StaticServiceCallServiceDiscoveryConfiguration discovery2 = (StaticServiceCallServiceDiscoveryConfiguration)conf2.getServiceDiscoveryConfiguration();
-        assertEquals(2, discovery2.getServers().size());
-        assertEquals("localhost:9092", discovery2.getServers().get(0));
-        assertEquals("localhost:9093,localhost:9094", discovery2.getServers().get(1));
+        MultiServiceCallServiceDiscoveryConfiguration discovery2 = (MultiServiceCallServiceDiscoveryConfiguration)conf2.getServiceDiscoveryConfiguration();
+        assertEquals(2, discovery2.getServiceDiscoveryConfigurations().size());
+        assertTrue(discovery2.getServiceDiscoveryConfigurations().get(0) instanceof StaticServiceCallServiceDiscoveryConfiguration);
+        assertTrue(discovery2.getServiceDiscoveryConfigurations().get(1) instanceof StaticServiceCallServiceDiscoveryConfiguration);
+
+        StaticServiceCallServiceDiscoveryConfiguration sconf1 = (StaticServiceCallServiceDiscoveryConfiguration)discovery2.getServiceDiscoveryConfigurations().get(0);
+        assertEquals(1, sconf1.getServers().size());
+        assertEquals("localhost:9092", sconf1.getServers().get(0));
+
+        StaticServiceCallServiceDiscoveryConfiguration sconf2 = (StaticServiceCallServiceDiscoveryConfiguration)discovery2.getServiceDiscoveryConfigurations().get(1);
+        assertEquals(1, sconf2.getServers().size());
+        assertEquals("localhost:9093,localhost:9094", sconf2.getServers().get(0));
     }
 
     protected SpringCamelContext createContext(String classpathConfigFile) {

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/components/camel-spring/src/test/resources/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.xml
----------------------------------------------------------------------
diff --git a/components/camel-spring/src/test/resources/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.xml b/components/camel-spring/src/test/resources/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.xml
index efc7711..d54348c 100644
--- a/components/camel-spring/src/test/resources/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.xml
+++ b/components/camel-spring/src/test/resources/org/apache/camel/spring/cloud/ServiceCallConfigurationTest.xml
@@ -32,10 +32,14 @@
     </serviceCallConfiguration>
 
     <serviceCallConfiguration id="conf2">
-      <staticServiceDiscovery>
-        <servers>localhost:9092</servers>
-        <servers>localhost:9093,localhost:9094</servers>
-      </staticServiceDiscovery>
+      <multiServiceDiscovery>
+        <staticServiceDiscovery>
+          <servers>localhost:9092</servers>
+        </staticServiceDiscovery>
+        <staticServiceDiscovery>
+          <servers>localhost:9093,localhost:9094</servers>
+        </staticServiceDiscovery>
+      </multiServiceDiscovery>
     </serviceCallConfiguration>
 
     <route id="test1">


[2/2] camel git commit: CAMEL-10710: ServiceCall EIP : improve Java DSL

Posted by lb...@apache.org.
CAMEL-10710: ServiceCall EIP : improve Java DSL


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

Branch: refs/heads/master
Commit: 80f746dfd38da6e5a4812de0fccfcaefd07e9edd
Parents: fdb508a
Author: lburgazzoli <lb...@gmail.com>
Authored: Mon Jan 16 17:08:03 2017 +0100
Committer: lburgazzoli <lb...@gmail.com>
Committed: Mon Jan 16 17:08:03 2017 +0100

----------------------------------------------------------------------
 .../impl/cloud/CachingServiceDiscovery.java     |  10 +-
 .../cloud/CachingServiceDiscoveryFactory.java   |  78 ++++++++
 .../camel/impl/cloud/MultiServiceDiscovery.java |  59 ++++++
 .../cloud/MultiServiceDiscoveryFactory.java     |  54 ++++++
 .../impl/cloud/StaticServiceDiscovery.java      |   9 +-
 ...erviceCallServiceDiscoveryConfiguration.java | 190 +++++++++++++++++++
 ...erviceCallServiceDiscoveryConfiguration.java | 169 +++++++++++++++++
 .../ServiceCallConfigurationDefinition.java     |  66 +++++++
 .../model/cloud/ServiceCallDefinition.java      |  68 +++++++
 .../ServiceCallExpressionConfiguration.java     |  26 ++-
 .../ServiceCallLoadBalancerConfiguration.java   |  24 ++-
 .../ServiceCallServiceChooserConfiguration.java |  22 ++-
 ...erviceCallServiceDiscoveryConfiguration.java |  24 ++-
 .../ServiceCallServiceFilterConfiguration.java  |  24 ++-
 ...erviceCallServiceDiscoveryConfiguration.java |  19 ++
 .../camel/cloud/caching-service-discovery       |  17 ++
 .../apache/camel/cloud/multi-service-discovery  |  17 ++
 .../org/apache/camel/model/cloud/jaxb.index     |   2 +
 ...chingServiceCallServiceListStrategyTest.java |  56 ------
 .../impl/cloud/CachingServiceDiscoveryTest.java |  67 +++++++
 .../impl/cloud/MultiServiceDiscoveryTest.java   |  71 +++++++
 .../ConsulDefaultServiceCallRouteTest.java      | 112 +++++++++++
 .../cloud/ConsulRibbonServiceCallRouteTest.java | 114 +++++++++++
 .../cloud/ConsulServiceCallRouteTest.java       | 118 ------------
 .../etcd/cloud/EtcdServiceCallRouteTest.java    |  11 +-
 .../cloud/RibbonServiceCallRouteDslTest.java    |  65 +++++++
 .../cloud/ServiceCallConfigurationTest.java     |  18 +-
 .../cloud/ServiceCallConfigurationTest.xml      |  12 +-
 28 files changed, 1293 insertions(+), 229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscovery.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscovery.java
index 39394bb..56ee5fb 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscovery.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscovery.java
@@ -33,10 +33,18 @@ public class CachingServiceDiscovery implements ServiceDiscovery {
     private long timeout;
 
     public CachingServiceDiscovery(ServiceDiscovery delegate) {
+        this(delegate, 60 * 1000);
+    }
+
+    public CachingServiceDiscovery(ServiceDiscovery delegate, long timeout) {
         this.delegate = ObjectHelper.notNull(delegate, "delegate");
         this.lastUpdate = 0;
         this.services = Collections.emptyList();
-        this.timeout = 60 * 1000; // 1 min;
+        this.timeout = timeout;
+    }
+
+    public ServiceDiscovery getDelegate() {
+        return this.delegate;
     }
 
     public void setTimeout(long timeout) {

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryFactory.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryFactory.java
new file mode 100644
index 0000000..7accae3
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryFactory.java
@@ -0,0 +1,78 @@
+/**
+ * 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.impl.cloud;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.cloud.ServiceDiscovery;
+import org.apache.camel.cloud.ServiceDiscoveryFactory;
+import org.apache.camel.util.ObjectHelper;
+
+public class CachingServiceDiscoveryFactory implements ServiceDiscoveryFactory {
+    private Integer timeout;
+    private TimeUnit units;
+    private ServiceDiscovery serviceDiscovery;
+
+    public CachingServiceDiscoveryFactory() {
+    }
+
+    // *************************************************************************
+    // Properties
+    // *************************************************************************
+
+    public Integer getTimeout() {
+        return timeout;
+    }
+
+    public void setTimeout(Integer timeout) {
+        this.timeout = timeout;
+    }
+
+    public TimeUnit getUnits() {
+        return units;
+    }
+
+    public void setUnits(TimeUnit units) {
+        this.units = units;
+    }
+
+    public ServiceDiscovery getServiceDiscovery() {
+        return serviceDiscovery;
+    }
+
+    public void setServiceDiscovery(ServiceDiscovery serviceDiscovery) {
+        this.serviceDiscovery = serviceDiscovery;
+    }
+
+    // *************************************************************************
+    // Factory
+    // *************************************************************************
+
+    @Override
+    public ServiceDiscovery newInstance(CamelContext camelContext) throws Exception {
+        ObjectHelper.notNull(serviceDiscovery, "ServiceDiscovery configuration");
+        ObjectHelper.notNull(timeout, "CachingServiceDiscovery timeout");
+        ObjectHelper.notNull(units, "CachingServiceDiscovery time units");
+
+        return new CachingServiceDiscovery(
+            serviceDiscovery,
+            units.toMillis(timeout)
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscovery.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscovery.java
new file mode 100644
index 0000000..44ea721
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscovery.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.impl.cloud;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.camel.cloud.ServiceDefinition;
+import org.apache.camel.cloud.ServiceDiscovery;
+
+public class MultiServiceDiscovery implements ServiceDiscovery {
+    private final List<ServiceDiscovery> delegates;
+
+    public MultiServiceDiscovery(List<ServiceDiscovery> delegates) {
+        this.delegates = Collections.unmodifiableList(delegates);
+    }
+
+    public List<ServiceDiscovery> getDelegates() {
+        return this.delegates;
+    }
+
+    @Override
+    public List<ServiceDefinition> getInitialListOfServices(String name) {
+        return delegates.stream()
+            .flatMap(d -> d.getInitialListOfServices(name).stream())
+            .collect(Collectors.toList());
+    }
+
+    @Override
+    public List<ServiceDefinition> getUpdatedListOfServices(String name) {
+        return delegates.stream()
+            .flatMap(d -> d.getInitialListOfServices(name).stream())
+            .collect(Collectors.toList());
+    }
+
+    // **********************
+    // Helpers
+    // **********************
+
+    public static MultiServiceDiscovery wrap(ServiceDiscovery... delegates) {
+        return new MultiServiceDiscovery(Arrays.asList(delegates));
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryFactory.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryFactory.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryFactory.java
new file mode 100644
index 0000000..61dcd76
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryFactory.java
@@ -0,0 +1,54 @@
+/**
+ * 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.impl.cloud;
+
+import java.util.List;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.cloud.ServiceDiscovery;
+import org.apache.camel.cloud.ServiceDiscoveryFactory;
+import org.apache.camel.util.ObjectHelper;
+
+public class MultiServiceDiscoveryFactory implements ServiceDiscoveryFactory {
+    private List<ServiceDiscovery> serviceDiscoveryList;
+
+    public MultiServiceDiscoveryFactory() {
+    }
+
+    // *************************************************************************
+    // Properties
+    // *************************************************************************
+
+    public List<ServiceDiscovery> getServiceDiscoveryList() {
+        return serviceDiscoveryList;
+    }
+
+    public void setServiceDiscoveryList(List<ServiceDiscovery> serviceDiscoveryList) {
+        this.serviceDiscoveryList = serviceDiscoveryList;
+    }
+
+    // *************************************************************************
+    // Factory
+    // *************************************************************************
+
+    @Override
+    public ServiceDiscovery newInstance(CamelContext camelContext) throws Exception {
+        ObjectHelper.notNull(serviceDiscoveryList, "ServiceDiscovery list");
+
+        return new MultiServiceDiscovery(serviceDiscoveryList);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/impl/cloud/StaticServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/cloud/StaticServiceDiscovery.java b/camel-core/src/main/java/org/apache/camel/impl/cloud/StaticServiceDiscovery.java
index 83fccc5..49d3094 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/cloud/StaticServiceDiscovery.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/cloud/StaticServiceDiscovery.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 import org.apache.camel.cloud.ServiceDefinition;
 import org.apache.camel.util.StringHelper;
@@ -77,7 +78,7 @@ public class StaticServiceDiscovery extends DefaultServiceDiscovery {
         for (String part : parts) {
             String service = StringHelper.before(part, "@");
             if (service != null) {
-                part = StringHelper.after(service, "@");
+                part = StringHelper.after(part, "@");
             }
             String host = StringHelper.before(part, ":");
             String port = StringHelper.after(part, ":");
@@ -127,6 +128,10 @@ public class StaticServiceDiscovery extends DefaultServiceDiscovery {
 
     @Override
     public List<ServiceDefinition> getUpdatedListOfServices(String name) {
-        return Collections.unmodifiableList(servers);
+        return Collections.unmodifiableList(
+            servers.stream()
+                .filter(s -> Objects.isNull(s.getName()) || Objects.equals(name, s.getName()))
+                .collect(Collectors.toList())
+        );
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/model/cloud/CachingServiceCallServiceDiscoveryConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/CachingServiceCallServiceDiscoveryConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/CachingServiceCallServiceDiscoveryConfiguration.java
new file mode 100644
index 0000000..5c1682b
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/CachingServiceCallServiceDiscoveryConfiguration.java
@@ -0,0 +1,190 @@
+/**
+ * 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;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+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 javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.xml.TimeUnitAdapter;
+import org.apache.camel.spi.Metadata;
+
+@Metadata(label = "routing,cloud,service-discovery")
+@XmlRootElement(name = "cachingServiceDiscovery")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CachingServiceCallServiceDiscoveryConfiguration extends ServiceCallServiceDiscoveryConfiguration {
+    @XmlAttribute @Metadata(defaultValue = "60")
+    private Integer timeout = 60;
+    @XmlAttribute
+    @XmlJavaTypeAdapter(TimeUnitAdapter.class) @Metadata(defaultValue = "SECONDS")
+    private TimeUnit units = TimeUnit.SECONDS;
+    @XmlElements({
+        @XmlElement(name = "consulServiceDiscovery", type = ConsulServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "dnsServiceDiscovery", type = DnsServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "etcdServiceDiscovery", type = EtcdServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "kubernetesServiceDiscovery", type = KubernetesServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "multiServiceDiscovery", type = MultiServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "staticServiceDiscovery", type = StaticServiceCallServiceDiscoveryConfiguration.class)}
+    )
+    private ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration;
+
+    public CachingServiceCallServiceDiscoveryConfiguration() {
+        this(null);
+    }
+
+    public CachingServiceCallServiceDiscoveryConfiguration(ServiceCallDefinition parent) {
+        super(parent, "caching-service-discovery");
+    }
+
+    // *************************************************************************
+    // Properties
+    // *************************************************************************
+
+    public Integer getTimeout() {
+        return timeout;
+    }
+
+    /**
+     * Set the time the services will be retained.
+     */
+    public void setTimeout(Integer timeout) {
+        this.timeout = timeout;
+    }
+
+    public TimeUnit getUnits() {
+        return units;
+    }
+
+    /**
+     * Set the time unit for the timeout.
+     */
+    public void setUnits(TimeUnit units) {
+        this.units = units;
+    }
+
+    public ServiceCallServiceDiscoveryConfiguration getServiceDiscoveryConfiguration() {
+        return serviceDiscoveryConfiguration;
+    }
+
+    /**
+     * Set the service-call configuration to use
+     */
+    public void setServiceDiscoveryConfiguration(ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration) {
+        this.serviceDiscoveryConfiguration = serviceDiscoveryConfiguration;
+    }
+
+
+    // *************************************************************************
+    // Fluent API
+    // *************************************************************************
+
+    /**
+     * Set the time the services will be retained.
+     */
+    public CachingServiceCallServiceDiscoveryConfiguration timeout(Integer timeout) {
+        setTimeout(timeout);
+        return this;
+    }
+
+    /**
+     *  Set the time unit for the timeout.
+     */
+    public CachingServiceCallServiceDiscoveryConfiguration units(TimeUnit units) {
+        setUnits(units);
+        return this;
+    }
+
+    /**
+     *  Set the service-call configuration to use
+     */
+    public CachingServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration(ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration) {
+        setServiceDiscoveryConfiguration(serviceDiscoveryConfiguration);
+        return this;
+    }
+
+    // *****************************
+    // Shortcuts - ServiceDiscovery
+    // *****************************
+
+    public CachingServiceCallServiceDiscoveryConfiguration cachingServiceDiscovery() {
+        CachingServiceCallServiceDiscoveryConfiguration conf = new CachingServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return serviceDiscoveryConfiguration(conf);
+    }
+
+    public ConsulServiceCallServiceDiscoveryConfiguration consulServiceDiscovery() {
+        ConsulServiceCallServiceDiscoveryConfiguration conf = new ConsulServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public DnsServiceCallServiceDiscoveryConfiguration dnsServiceDiscovery() {
+        DnsServiceCallServiceDiscoveryConfiguration conf = new DnsServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public EtcdServiceCallServiceDiscoveryConfiguration etcdServiceDiscovery() {
+        EtcdServiceCallServiceDiscoveryConfiguration conf = new EtcdServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public KubernetesServiceCallServiceDiscoveryConfiguration kubernetesServiceDiscovery() {
+        KubernetesServiceCallServiceDiscoveryConfiguration conf = new KubernetesServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public MultiServiceCallServiceDiscoveryConfiguration multiServiceDiscovery() {
+        MultiServiceCallServiceDiscoveryConfiguration conf = new MultiServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public StaticServiceCallServiceDiscoveryConfiguration staticServiceDiscovery() {
+        StaticServiceCallServiceDiscoveryConfiguration conf = new StaticServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    @Override
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception {
+        if (serviceDiscoveryConfiguration != null) {
+            parameters.put("serviceDiscovery", serviceDiscoveryConfiguration.newInstance(camelContext));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/model/cloud/MultiServiceCallServiceDiscoveryConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/MultiServiceCallServiceDiscoveryConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/MultiServiceCallServiceDiscoveryConfiguration.java
new file mode 100644
index 0000000..b45ac99
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/MultiServiceCallServiceDiscoveryConfiguration.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElements;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.cloud.ServiceDiscovery;
+import org.apache.camel.spi.Metadata;
+
+@Metadata(label = "routing,cloud,service-discovery")
+@XmlRootElement(name = "multiServiceDiscovery")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class MultiServiceCallServiceDiscoveryConfiguration extends ServiceCallServiceDiscoveryConfiguration {
+    @XmlElements({
+        @XmlElement(name = "consulServiceDiscovery", type = ConsulServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "dnsServiceDiscovery", type = DnsServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "etcdServiceDiscovery", type = EtcdServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "kubernetesServiceDiscovery", type = KubernetesServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "staticServiceDiscovery", type = StaticServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "cachingServiceDiscovery", type = CachingServiceCallServiceDiscoveryConfiguration.class)}
+    )
+    private List<ServiceCallServiceDiscoveryConfiguration> serviceDiscoveryConfigurations;
+
+    public MultiServiceCallServiceDiscoveryConfiguration() {
+        this(null);
+    }
+
+    public MultiServiceCallServiceDiscoveryConfiguration(ServiceCallDefinition parent) {
+        super(parent, "multi-service-discovery");
+    }
+
+    // *************************************************************************
+    // Properties
+    // *************************************************************************
+
+    public List<ServiceCallServiceDiscoveryConfiguration> getServiceDiscoveryConfigurations() {
+        return serviceDiscoveryConfigurations;
+    }
+
+    /**
+     * List of ServiceDiscovery configuration to use
+     * @param serviceDiscoveryConfigurations
+     */
+    public void setServiceDiscoveryConfigurations(List<ServiceCallServiceDiscoveryConfiguration> serviceDiscoveryConfigurations) {
+        this.serviceDiscoveryConfigurations = serviceDiscoveryConfigurations;
+    }
+
+    /**
+     *  Add a ServiceDiscovery configuration
+     */
+    public void addServiceDiscoveryConfigurations(ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration) {
+        if (serviceDiscoveryConfigurations == null) {
+            serviceDiscoveryConfigurations = new ArrayList<>();
+        }
+
+        serviceDiscoveryConfigurations.add(serviceDiscoveryConfiguration);
+    }
+
+    // *************************************************************************
+    // Fluent API
+    // *************************************************************************
+
+    /**
+     *  List of ServiceDiscovery configuration to use
+     */
+    public MultiServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfigurations(List<ServiceCallServiceDiscoveryConfiguration> serviceDiscoveryConfigurations) {
+        setServiceDiscoveryConfigurations(serviceDiscoveryConfigurations);
+        return this;
+    }
+
+    /**
+     *  Add a ServiceDiscovery configuration
+     */
+    public MultiServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration(ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration) {
+        addServiceDiscoveryConfigurations(serviceDiscoveryConfiguration);
+        return this;
+    }
+
+    // *****************************
+    // Shortcuts - ServiceDiscovery
+    // *****************************
+
+    public CachingServiceCallServiceDiscoveryConfiguration cachingServiceDiscovery() {
+        CachingServiceCallServiceDiscoveryConfiguration conf = new CachingServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public ConsulServiceCallServiceDiscoveryConfiguration consulServiceDiscovery() {
+        ConsulServiceCallServiceDiscoveryConfiguration conf = new ConsulServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public DnsServiceCallServiceDiscoveryConfiguration dnsServiceDiscovery() {
+        DnsServiceCallServiceDiscoveryConfiguration conf = new DnsServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public EtcdServiceCallServiceDiscoveryConfiguration etcdServiceDiscovery() {
+        EtcdServiceCallServiceDiscoveryConfiguration conf = new EtcdServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public KubernetesServiceCallServiceDiscoveryConfiguration kubernetesServiceDiscovery() {
+        KubernetesServiceCallServiceDiscoveryConfiguration conf = new KubernetesServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public MultiServiceCallServiceDiscoveryConfiguration multiServiceDiscovery() {
+        MultiServiceCallServiceDiscoveryConfiguration conf = new MultiServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    public StaticServiceCallServiceDiscoveryConfiguration staticServiceDiscovery() {
+        StaticServiceCallServiceDiscoveryConfiguration conf = new StaticServiceCallServiceDiscoveryConfiguration();
+        addServiceDiscoveryConfigurations(conf);
+
+        return conf;
+    }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    @Override
+    protected void postProcessFactoryParameters(final CamelContext camelContext, final Map<String, Object> parameters) throws Exception {
+        if (serviceDiscoveryConfigurations != null && !serviceDiscoveryConfigurations.isEmpty()) {
+            List<ServiceDiscovery> discoveries = new ArrayList<>(serviceDiscoveryConfigurations.size());
+            for (ServiceCallServiceDiscoveryConfiguration conf : serviceDiscoveryConfigurations) {
+                discoveries.add(conf.newInstance(camelContext));
+            }
+
+            parameters.put("serviceDiscoveryList", discoveries);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 c35fed9..b4aaac0 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
@@ -67,10 +67,12 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
     @XmlTransient
     private Expression expression;
     @XmlElements({
+        @XmlElement(name = "cachingServiceDiscovery", type = CachingServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "consulServiceDiscovery", type = ConsulServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "dnsServiceDiscovery", type = DnsServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "etcdServiceDiscovery", type = EtcdServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "kubernetesServiceDiscovery", type = KubernetesServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "multiServiceDiscovery", type = MultiServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "staticServiceDiscovery", type = StaticServiceCallServiceDiscoveryConfiguration.class)}
     )
     private ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration;
@@ -404,4 +406,68 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
         setExpressionConfiguration(expressionConfiguration);
         return this;
     }
+
+    // *****************************
+    // Shortcuts - ServiceDiscovery
+    // *****************************
+
+    public CachingServiceCallServiceDiscoveryConfiguration cachingServiceDiscovery() {
+        CachingServiceCallServiceDiscoveryConfiguration conf = new CachingServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public ConsulServiceCallServiceDiscoveryConfiguration consulServiceDiscovery() {
+        ConsulServiceCallServiceDiscoveryConfiguration conf = new ConsulServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public DnsServiceCallServiceDiscoveryConfiguration dnsServiceDiscovery() {
+        DnsServiceCallServiceDiscoveryConfiguration conf = new DnsServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public EtcdServiceCallServiceDiscoveryConfiguration etcdServiceDiscovery() {
+        EtcdServiceCallServiceDiscoveryConfiguration conf = new EtcdServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public KubernetesServiceCallServiceDiscoveryConfiguration kubernetesServiceDiscovery() {
+        KubernetesServiceCallServiceDiscoveryConfiguration conf = new KubernetesServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public MultiServiceCallServiceDiscoveryConfiguration multiServiceDiscovery() {
+        MultiServiceCallServiceDiscoveryConfiguration conf = new MultiServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public StaticServiceCallServiceDiscoveryConfiguration staticServiceDiscovery() {
+        StaticServiceCallServiceDiscoveryConfiguration conf = new StaticServiceCallServiceDiscoveryConfiguration();
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    // *****************************
+    // Shortcuts - LoadBalancer
+    // *****************************
+
+    public RibbonServiceCallLoadBalancerConfiguration ribbonLoadBalancer() {
+        RibbonServiceCallLoadBalancerConfiguration conf = new RibbonServiceCallLoadBalancerConfiguration();
+        setLoadBalancerConfiguration(conf);
+
+        return conf;
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 84a0fd7..caa97ff 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
@@ -91,10 +91,12 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
     private Expression expression;
 
     @XmlElements({
+        @XmlElement(name = "cachingServiceDiscovery", type = CachingServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "consulServiceDiscovery", type = ConsulServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "dnsServiceDiscovery", type = DnsServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "etcdServiceDiscovery", type = EtcdServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "kubernetesServiceDiscovery", type = KubernetesServiceCallServiceDiscoveryConfiguration.class),
+        @XmlElement(name = "multiServiceDiscovery", type = MultiServiceCallServiceDiscoveryConfiguration.class),
         @XmlElement(name = "staticServiceDiscovery", type = StaticServiceCallServiceDiscoveryConfiguration.class)}
     )
     private ServiceCallServiceDiscoveryConfiguration serviceDiscoveryConfiguration;
@@ -474,6 +476,72 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
     }
 
     // *****************************
+    // Shortcuts - ServiceDiscovery
+    // *****************************
+
+    /*
+    public CachingServiceCallServiceDiscoveryConfiguration cachingServiceDiscovery() {
+        CachingServiceCallServiceDiscoveryConfiguration conf = new CachingServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+    */
+
+    public ConsulServiceCallServiceDiscoveryConfiguration consulServiceDiscovery() {
+        ConsulServiceCallServiceDiscoveryConfiguration conf = new ConsulServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public DnsServiceCallServiceDiscoveryConfiguration dnsServiceDiscovery() {
+        DnsServiceCallServiceDiscoveryConfiguration conf = new DnsServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public EtcdServiceCallServiceDiscoveryConfiguration etcdServiceDiscovery() {
+        EtcdServiceCallServiceDiscoveryConfiguration conf = new EtcdServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public KubernetesServiceCallServiceDiscoveryConfiguration kubernetesServiceDiscovery() {
+        KubernetesServiceCallServiceDiscoveryConfiguration conf = new KubernetesServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public MultiServiceCallServiceDiscoveryConfiguration multiServiceDiscovery() {
+        MultiServiceCallServiceDiscoveryConfiguration conf = new MultiServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    public StaticServiceCallServiceDiscoveryConfiguration staticServiceDiscovery() {
+        StaticServiceCallServiceDiscoveryConfiguration conf = new StaticServiceCallServiceDiscoveryConfiguration(this);
+        setServiceDiscoveryConfiguration(conf);
+
+        return conf;
+    }
+
+    // *****************************
+    // Shortcuts - LoadBalancer
+    // *****************************
+
+    public RibbonServiceCallLoadBalancerConfiguration ribbonLoadBalancer() {
+        RibbonServiceCallLoadBalancerConfiguration conf = new RibbonServiceCallLoadBalancerConfiguration(this);
+        setLoadBalancerConfiguration(conf);
+
+        return conf;
+    }
+
+    // *****************************
     // Processor Factory
     // *****************************
 

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 0b66595..873c9e3 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
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -44,13 +43,13 @@ import org.apache.camel.util.IntrospectionSupport;
 import org.apache.camel.util.ObjectHelper;
 
 @Metadata(label = "routing,cloud")
-@XmlRootElement(name = "expressionConfiguration")
+@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 Optional<ServiceCallDefinition> parent;
+    private final ServiceCallDefinition parent;
     @XmlTransient
     private final String factoryKey;
     @XmlElement(name = "properties") @Metadata(label = "advanced")
@@ -65,13 +64,16 @@ public class ServiceCallExpressionConfiguration extends IdentifiedType implement
     }
 
     public ServiceCallExpressionConfiguration(ServiceCallDefinition parent, String factoryKey) {
-        this.parent = Optional.ofNullable(parent);
+        this.parent = parent;
         this.factoryKey = factoryKey;
     }
 
-    public ProcessorDefinition end() {
-        // end parent as well so we do not have to use 2x end
-        return this.parent.orElseGet(null);
+    public ServiceCallDefinition end() {
+        return this.parent;
+    }
+
+    public ProcessorDefinition<?> endParent() {
+        return this.parent.end();
     }
 
     // *************************************************************************
@@ -207,8 +209,9 @@ public class ServiceCallExpressionConfiguration extends IdentifiedType implement
                 IntrospectionSupport.getProperties(this, parameters, null, false);
                 parameters.put("properties", getPropertiesAsMap(camelContext));
 
-                IntrospectionSupport.setProperties(factory, parameters);
+                postProcessFactoryParameters(camelContext, parameters);
 
+                IntrospectionSupport.setProperties(factory, parameters);
 
                 answer = factory.newInstance(camelContext);
             } catch (Exception e) {
@@ -218,4 +221,11 @@ public class ServiceCallExpressionConfiguration extends IdentifiedType implement
 
         return answer;
     }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception  {
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallLoadBalancerConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallLoadBalancerConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallLoadBalancerConfiguration.java
index 8739a85..42f644a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallLoadBalancerConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/ServiceCallLoadBalancerConfiguration.java
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -48,7 +47,7 @@ public class ServiceCallLoadBalancerConfiguration extends IdentifiedType impleme
     private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
 
     @XmlTransient
-    private final Optional<ServiceCallDefinition> parent;
+    private final ServiceCallDefinition parent;
     @XmlTransient
     private final String factoryKey;
     @XmlElement(name = "properties") @Metadata(label = "advanced")
@@ -59,13 +58,16 @@ public class ServiceCallLoadBalancerConfiguration extends IdentifiedType impleme
     }
 
     public ServiceCallLoadBalancerConfiguration(ServiceCallDefinition parent, String factoryKey) {
-        this.parent = Optional.ofNullable(parent);
+        this.parent = parent;
         this.factoryKey = factoryKey;
     }
 
-    public ProcessorDefinition end() {
-        // end parent as well so we do not have to use 2x end
-        return this.parent.orElseGet(null);
+    public ServiceCallDefinition end() {
+        return this.parent;
+    }
+
+    public ProcessorDefinition<?> endParent() {
+        return this.parent.end();
     }
 
     // *************************************************************************
@@ -163,8 +165,9 @@ public class ServiceCallLoadBalancerConfiguration extends IdentifiedType impleme
                 IntrospectionSupport.getProperties(this, parameters, null, false);
                 parameters.put("properties", getPropertiesAsMap(camelContext));
 
-                IntrospectionSupport.setProperties(factory, parameters);
+                postProcessFactoryParameters(camelContext, parameters);
 
+                IntrospectionSupport.setProperties(factory, parameters);
 
                 answer = factory.newInstance(camelContext);
             } catch (Exception e) {
@@ -174,4 +177,11 @@ public class ServiceCallLoadBalancerConfiguration extends IdentifiedType impleme
 
         return answer;
     }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception  {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 d3a418b..0a0d8f4 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
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -48,7 +47,7 @@ public class ServiceCallServiceChooserConfiguration extends IdentifiedType imple
     private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
 
     @XmlTransient
-    private final Optional<ServiceCallDefinition> parent;
+    private final ServiceCallDefinition parent;
     @XmlTransient
     private final String factoryKey;
     @XmlElement(name = "properties") @Metadata(label = "advanced")
@@ -59,13 +58,16 @@ public class ServiceCallServiceChooserConfiguration extends IdentifiedType imple
     }
 
     public ServiceCallServiceChooserConfiguration(ServiceCallDefinition parent, String factoryKey) {
-        this.parent = Optional.ofNullable(parent);
+        this.parent = parent;
         this.factoryKey = factoryKey;
     }
 
-    public ProcessorDefinition end() {
-        // end parent as well so we do not have to use 2x end
-        return this.parent.orElseGet(null);
+    public ServiceCallDefinition end() {
+        return this.parent;
+    }
+
+    public ProcessorDefinition<?> endParent() {
+        return this.parent.end();
     }
 
     // *************************************************************************
@@ -165,7 +167,6 @@ public class ServiceCallServiceChooserConfiguration extends IdentifiedType imple
 
                 IntrospectionSupport.setProperties(factory, parameters);
 
-
                 answer = factory.newInstance(camelContext);
             } catch (Exception e) {
                 throw new IllegalArgumentException(e);
@@ -174,4 +175,11 @@ public class ServiceCallServiceChooserConfiguration extends IdentifiedType imple
 
         return answer;
     }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception  {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 600705b..fed3134 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
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -48,7 +47,7 @@ public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType imp
     private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
 
     @XmlTransient
-    private final Optional<ServiceCallDefinition> parent;
+    private final ServiceCallDefinition parent;
     @XmlTransient
     private final String factoryKey;
     @XmlElement(name = "properties") @Metadata(label = "advanced")
@@ -59,13 +58,16 @@ public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType imp
     }
 
     public ServiceCallServiceDiscoveryConfiguration(ServiceCallDefinition parent, String factoryKey) {
-        this.parent = Optional.ofNullable(parent);
+        this.parent = parent;
         this.factoryKey = factoryKey;
     }
 
-    public ProcessorDefinition end() {
-        // end parent as well so we do not have to use 2x end
-        return this.parent.orElseGet(null);
+    public ServiceCallDefinition end() {
+        return this.parent;
+    }
+
+    public ProcessorDefinition<?> endParent() {
+        return this.parent.end();
     }
 
     // *************************************************************************
@@ -163,8 +165,9 @@ public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType imp
                 IntrospectionSupport.getProperties(this, parameters, null, false);
                 parameters.put("properties", getPropertiesAsMap(camelContext));
 
-                IntrospectionSupport.setProperties(factory, parameters);
+                postProcessFactoryParameters(camelContext, parameters);
 
+                IntrospectionSupport.setProperties(factory, parameters);
 
                 answer = factory.newInstance(camelContext);
             } catch (Exception e) {
@@ -174,4 +177,11 @@ public class ServiceCallServiceDiscoveryConfiguration extends IdentifiedType imp
 
         return answer;
     }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception  {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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 09651fa..6b6de0c 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
@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -48,7 +47,7 @@ public class ServiceCallServiceFilterConfiguration extends IdentifiedType implem
     private static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/cloud/";
 
     @XmlTransient
-    private final Optional<ServiceCallDefinition> parent;
+    private final ServiceCallDefinition parent;
     @XmlTransient
     private final String factoryKey;
     @XmlElement(name = "properties") @Metadata(label = "advanced")
@@ -59,13 +58,16 @@ public class ServiceCallServiceFilterConfiguration extends IdentifiedType implem
     }
 
     public ServiceCallServiceFilterConfiguration(ServiceCallDefinition parent, String factoryKey) {
-        this.parent = Optional.ofNullable(parent);
+        this.parent = parent;
         this.factoryKey = factoryKey;
     }
 
-    public ProcessorDefinition end() {
-        // end parent as well so we do not have to use 2x end
-        return this.parent.orElseGet(null);
+    public ServiceCallDefinition end() {
+        return this.parent;
+    }
+
+    public ProcessorDefinition<?> endParent() {
+        return this.parent.end();
     }
 
     // *************************************************************************
@@ -163,8 +165,9 @@ public class ServiceCallServiceFilterConfiguration extends IdentifiedType implem
                 IntrospectionSupport.getProperties(this, parameters, null, false);
                 parameters.put("properties", getPropertiesAsMap(camelContext));
 
-                IntrospectionSupport.setProperties(factory, parameters);
+                postProcessFactoryParameters(camelContext, parameters);
 
+                IntrospectionSupport.setProperties(factory, parameters);
 
                 answer = factory.newInstance(camelContext);
             } catch (Exception e) {
@@ -174,4 +177,11 @@ public class ServiceCallServiceFilterConfiguration extends IdentifiedType implem
 
         return answer;
     }
+
+    // *************************************************************************
+    // Utilities
+    // *************************************************************************
+
+    protected void postProcessFactoryParameters(CamelContext camelContext, Map<String, Object> parameters) throws Exception  {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/java/org/apache/camel/model/cloud/StaticServiceCallServiceDiscoveryConfiguration.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/cloud/StaticServiceCallServiceDiscoveryConfiguration.java b/camel-core/src/main/java/org/apache/camel/model/cloud/StaticServiceCallServiceDiscoveryConfiguration.java
index 0c5dae1..ce50c2f 100644
--- a/camel-core/src/main/java/org/apache/camel/model/cloud/StaticServiceCallServiceDiscoveryConfiguration.java
+++ b/camel-core/src/main/java/org/apache/camel/model/cloud/StaticServiceCallServiceDiscoveryConfiguration.java
@@ -17,6 +17,7 @@
 
 package org.apache.camel.model.cloud;
 
+import java.util.ArrayList;
 import java.util.List;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -55,6 +56,17 @@ public class StaticServiceCallServiceDiscoveryConfiguration extends ServiceCallS
         this.servers = servers;
     }
 
+    /**
+     * Sets the server list;
+     */
+    public void addServer(String server) {
+        if (this.servers == null) {
+            this.servers = new ArrayList<>();
+        }
+
+        this.servers.add(server);
+    }
+
     // *************************************************************************
     // Fluent API
     // *************************************************************************
@@ -66,4 +78,11 @@ public class StaticServiceCallServiceDiscoveryConfiguration extends ServiceCallS
         setServers(servers);
         return this;
     }
+    /**
+     * Add a server to the list of servers
+     */
+    public StaticServiceCallServiceDiscoveryConfiguration server(String server) {
+        addServer(server);
+        return this;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/caching-service-discovery
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/caching-service-discovery b/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/caching-service-discovery
new file mode 100644
index 0000000..7a57359
--- /dev/null
+++ b/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/caching-service-discovery
@@ -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.impl.cloud.CachingServiceDiscoveryFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/multi-service-discovery
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/multi-service-discovery b/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/multi-service-discovery
new file mode 100644
index 0000000..eacee81
--- /dev/null
+++ b/camel-core/src/main/resources/META-INF/services/org/apache/camel/cloud/multi-service-discovery
@@ -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.impl.cloud.MultiServiceDiscoveryFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/main/resources/org/apache/camel/model/cloud/jaxb.index
----------------------------------------------------------------------
diff --git a/camel-core/src/main/resources/org/apache/camel/model/cloud/jaxb.index b/camel-core/src/main/resources/org/apache/camel/model/cloud/jaxb.index
index e57f7e8..c275033 100644
--- a/camel-core/src/main/resources/org/apache/camel/model/cloud/jaxb.index
+++ b/camel-core/src/main/resources/org/apache/camel/model/cloud/jaxb.index
@@ -20,9 +20,11 @@ ServiceCallServiceDiscoveryConfiguration
 ServiceCallServiceFilterConfiguration
 ServiceCallServiceChooserConfiguration
 ServiceCallLoadBalancerConfiguration
+CachingServiceCallServiceDiscoveryConfiguration
 ConsulServiceCallServiceDiscoveryConfiguration
 DnsServiceCallServiceDiscoveryConfiguration
 EtcdServiceCallServiceDiscoveryConfiguration
 KubernetesServiceCallServiceDiscoveryConfiguration
+MultiServiceCallServiceDiscoveryConfiguration
 StaticServiceCallServiceDiscoveryConfiguration
 RibbonServiceCallLoadBalancerConfiguration

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceCallServiceListStrategyTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceCallServiceListStrategyTest.java b/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceCallServiceListStrategyTest.java
deleted file mode 100644
index f17c000..0000000
--- a/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceCallServiceListStrategyTest.java
+++ /dev/null
@@ -1,56 +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.impl.cloud;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.camel.cloud.ServiceDefinition;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CachingServiceCallServiceListStrategyTest {
-    @Test
-    public void testCachingServiceDiscovery() throws Exception {
-        MyStrategy strategy = new MyStrategy();
-        CachingServiceDiscovery caching = CachingServiceDiscovery.wrap(strategy, 1, TimeUnit.SECONDS);
-
-        strategy.addServer(new DefaultServiceDefinition("noname", "localhost", 1111));
-        Assert.assertEquals(1, caching.getUpdatedListOfServices("noname").size());
-        strategy.addServer(new DefaultServiceDefinition("noname", "localhost", 1112));
-        Assert.assertEquals(1, caching.getUpdatedListOfServices("noname").size());
-
-        // Let the cache expire
-        Thread.sleep(1100);
-
-        Assert.assertEquals(2, caching.getUpdatedListOfServices("noname").size());
-    }
-
-    private class MyStrategy extends DefaultServiceDiscovery {
-        private List<ServiceDefinition> services = new ArrayList<>();
-
-        @Override
-        public List<ServiceDefinition> getUpdatedListOfServices(String name) {
-            return services;
-        }
-
-        public void addServer(ServiceDefinition service) {
-            services.add(service);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryTest.java b/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryTest.java
new file mode 100644
index 0000000..661e8d7
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/impl/cloud/CachingServiceDiscoveryTest.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.impl.cloud;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.model.cloud.CachingServiceCallServiceDiscoveryConfiguration;
+import org.apache.camel.model.cloud.StaticServiceCallServiceDiscoveryConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CachingServiceDiscoveryTest extends ContextTestSupport {
+    @Test
+    public void testCachingServiceDiscovery() throws Exception {
+        StaticServiceDiscovery discovery = new StaticServiceDiscovery();
+        CachingServiceDiscovery caching = CachingServiceDiscovery.wrap(discovery, 1, TimeUnit.SECONDS);
+
+        discovery.addServer(new DefaultServiceDefinition("noname", "localhost", 1111));
+        Assert.assertEquals(1, caching.getUpdatedListOfServices("noname").size());
+        discovery.addServer(new DefaultServiceDefinition("noname", "localhost", 1112));
+        Assert.assertEquals(1, caching.getUpdatedListOfServices("noname").size());
+
+        // Let the cache expire
+        Thread.sleep(1100);
+
+        Assert.assertEquals(2, caching.getUpdatedListOfServices("noname").size());
+    }
+
+    @Test
+    public void testCachingServiceDiscoveryConfiguration() throws Exception {
+        StaticServiceCallServiceDiscoveryConfiguration staticConf = new StaticServiceCallServiceDiscoveryConfiguration();
+        staticConf.setServers(Arrays.asList("no-name@localhost:1111"));
+
+        CachingServiceCallServiceDiscoveryConfiguration cachingConf = new CachingServiceCallServiceDiscoveryConfiguration();
+        cachingConf.setServiceDiscoveryConfiguration(staticConf);
+        cachingConf.setTimeout(1);
+        cachingConf.setUnits(TimeUnit.SECONDS);
+
+        CachingServiceDiscovery caching = (CachingServiceDiscovery)cachingConf.newInstance(context);
+        StaticServiceDiscovery delegate = (StaticServiceDiscovery)caching.getDelegate();
+
+        Assert.assertEquals(1, caching.getUpdatedListOfServices("no-name").size());
+        delegate.addServer("no-name@localhost:1112");
+        Assert.assertEquals(1, caching.getUpdatedListOfServices("noname").size());
+
+        // Let the cache expire
+        Thread.sleep(1100);
+
+        Assert.assertEquals(2, caching.getUpdatedListOfServices("noname").size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/camel-core/src/test/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryTest.java b/camel-core/src/test/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryTest.java
new file mode 100644
index 0000000..108c5cd
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/impl/cloud/MultiServiceDiscoveryTest.java
@@ -0,0 +1,71 @@
+/**
+ * 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.impl.cloud;
+
+import java.util.Arrays;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.model.cloud.MultiServiceCallServiceDiscoveryConfiguration;
+import org.apache.camel.model.cloud.StaticServiceCallServiceDiscoveryConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MultiServiceDiscoveryTest extends ContextTestSupport {
+    @Test
+    public void testCachingServiceDiscovery() throws Exception {
+        StaticServiceDiscovery discovery1 = new StaticServiceDiscovery();
+        discovery1.addServer(new DefaultServiceDefinition("discovery1", "localhost", 1111));
+        discovery1.addServer(new DefaultServiceDefinition("discovery1", "localhost", 1112));
+
+        StaticServiceDiscovery discovery2 = new StaticServiceDiscovery();
+        discovery2.addServer(new DefaultServiceDefinition("discovery1", "localhost", 1113));
+        discovery2.addServer(new DefaultServiceDefinition("discovery2", "localhost", 1114));
+
+        MultiServiceDiscovery discovery = MultiServiceDiscovery.wrap(discovery1, discovery2);
+        Assert.assertEquals(3, discovery.getUpdatedListOfServices("discovery1").size());
+        Assert.assertEquals(1, discovery.getUpdatedListOfServices("discovery2").size());
+    }
+
+    @Test
+    public void testCachingServiceDiscoveryConfiguration() throws Exception {
+        StaticServiceCallServiceDiscoveryConfiguration staticConf1 = new StaticServiceCallServiceDiscoveryConfiguration();
+        staticConf1.setServers(Arrays.asList("discovery1@localhost:1111", "discovery1@localhost:1112"));
+
+        StaticServiceCallServiceDiscoveryConfiguration staticConf2 = new StaticServiceCallServiceDiscoveryConfiguration();
+        staticConf2.setServers(Arrays.asList("discovery1@localhost:1113", "discovery2@localhost:1114"));
+
+        MultiServiceCallServiceDiscoveryConfiguration multiConf = new MultiServiceCallServiceDiscoveryConfiguration();
+        multiConf.setServiceDiscoveryConfigurations(Arrays.asList(staticConf1, staticConf2));
+
+        MultiServiceDiscovery discovery = (MultiServiceDiscovery)multiConf.newInstance(context);
+        Assert.assertEquals(2, discovery.getDelegates().size());
+        Assert.assertEquals(3, discovery.getUpdatedListOfServices("discovery1").size());
+        Assert.assertEquals(1, discovery.getUpdatedListOfServices("discovery2").size());
+    }
+
+    @Test
+    public void testCachingServiceDiscoveryConfigurationDsl() throws Exception {
+        MultiServiceCallServiceDiscoveryConfiguration multiConf = new MultiServiceCallServiceDiscoveryConfiguration();
+        multiConf.staticServiceDiscovery().setServers(Arrays.asList("discovery1@localhost:1111", "discovery1@localhost:1112"));
+        multiConf.staticServiceDiscovery().setServers(Arrays.asList("discovery1@localhost:1113", "discovery2@localhost:1114"));
+
+        MultiServiceDiscovery discovery = (MultiServiceDiscovery)multiConf.newInstance(context);
+        Assert.assertEquals(2, discovery.getDelegates().size());
+        Assert.assertEquals(3, discovery.getUpdatedListOfServices("discovery1").size());
+        Assert.assertEquals(1, discovery.getUpdatedListOfServices("discovery2").size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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
new file mode 100644
index 0000000..749459d
--- /dev/null
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulDefaultServiceCallRouteTest.java
@@ -0,0 +1,112 @@
+/**
+ * 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.consul.cloud;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.orbitz.consul.AgentClient;
+import com.orbitz.consul.model.agent.ImmutableRegistration;
+import com.orbitz.consul.model.agent.Registration;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.consul.ConsulTestSupport;
+import org.junit.Test;
+
+public class ConsulDefaultServiceCallRouteTest extends ConsulTestSupport {
+    private static final String SERVICE_NAME = "http-service";
+    private static final int SERVICE_COUNT = 5;
+    private static final int SERVICE_PORT_BASE = 8080;
+
+    private AgentClient client;
+    private List<Registration> registrations;
+    private List<String> expectedBodies;
+
+    // *************************************************************************
+    // Setup / tear down
+    // *************************************************************************
+
+    @Override
+    protected void doPreSetup() throws Exception {
+        client = getConsul().agentClient();
+
+        registrations = new ArrayList<>(SERVICE_COUNT);
+        expectedBodies = new ArrayList<>(SERVICE_COUNT);
+
+        for (int i = 0; i < SERVICE_COUNT; i++) {
+            Registration r = ImmutableRegistration.builder()
+                .id("service-" + i)
+                .name(SERVICE_NAME)
+                .address("127.0.0.1")
+                .port(SERVICE_PORT_BASE + i)
+                .build();
+
+            client.register(r);
+
+            registrations.add(r);
+            expectedBodies.add("ping on " + r.getPort().get());
+        }
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+
+
+        registrations.forEach(r -> client.deregister(r.getId()));
+    }
+
+    // *************************************************************************
+    // Test
+    // *************************************************************************
+
+    @Test
+    public void testServiceCall() throws Exception {
+        getMockEndpoint("mock:result").expectedMessageCount(SERVICE_COUNT);
+        getMockEndpoint("mock:result").expectedBodiesReceivedInAnyOrder(expectedBodies);
+
+        registrations.forEach(r -> template.sendBody("direct:start", "ping"));
+
+        assertMockEndpointsSatisfied();
+    }
+
+    // *************************************************************************
+    // Route
+    // *************************************************************************
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .serviceCall()
+                        .name(SERVICE_NAME)
+                        .consulServiceDiscovery()
+                        .endParent()
+                    .to("log:org.apache.camel.component.consul.processor.service?level=INFO&showAll=true&multiline=true")
+                    .to("mock:result");
+
+                registrations.forEach(r ->
+                    fromF("jetty:http://%s:%d", r.getAddress().get(), r.getPort().get())
+                        .transform().simple("${in.body} on " + r.getPort().get())
+                );
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/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
new file mode 100644
index 0000000..e0e30f2
--- /dev/null
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulRibbonServiceCallRouteTest.java
@@ -0,0 +1,114 @@
+/**
+ * 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.consul.cloud;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.orbitz.consul.AgentClient;
+import com.orbitz.consul.model.agent.ImmutableRegistration;
+import com.orbitz.consul.model.agent.Registration;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.consul.ConsulTestSupport;
+import org.junit.Test;
+
+public class ConsulRibbonServiceCallRouteTest extends ConsulTestSupport {
+    private static final String SERVICE_NAME = "http-service";
+    private static final int SERVICE_COUNT = 5;
+    private static final int SERVICE_PORT_BASE = 8080;
+
+    private AgentClient client;
+    private List<Registration> registrations;
+    private List<String> expectedBodies;
+
+    // *************************************************************************
+    // Setup / tear down
+    // *************************************************************************
+
+    @Override
+    protected void doPreSetup() throws Exception {
+        client = getConsul().agentClient();
+
+        registrations = new ArrayList<>(SERVICE_COUNT);
+        expectedBodies = new ArrayList<>(SERVICE_COUNT);
+
+        for (int i = 0; i < SERVICE_COUNT; i++) {
+            Registration r = ImmutableRegistration.builder()
+                .id("service-" + i)
+                .name(SERVICE_NAME)
+                .address("127.0.0.1")
+                .port(SERVICE_PORT_BASE + i)
+                .build();
+
+            client.register(r);
+
+            registrations.add(r);
+            expectedBodies.add("ping on " + r.getPort().get());
+        }
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+
+
+        registrations.forEach(r -> client.deregister(r.getId()));
+    }
+
+    // *************************************************************************
+    // Test
+    // *************************************************************************
+
+    @Test
+    public void testServiceCall() throws Exception {
+        getMockEndpoint("mock:result").expectedMessageCount(SERVICE_COUNT);
+        getMockEndpoint("mock:result").expectedBodiesReceivedInAnyOrder(expectedBodies);
+
+        registrations.forEach(r -> template.sendBody("direct:start", "ping"));
+
+        assertMockEndpointsSatisfied();
+    }
+
+    // *************************************************************************
+    // Route
+    // *************************************************************************
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .serviceCall()
+                        .name(SERVICE_NAME)
+                        .ribbonLoadBalancer()
+                            .end()
+                        .consulServiceDiscovery()
+                            .endParent()
+                    .to("log:org.apache.camel.component.consul.processor.service?level=INFO&showAll=true&multiline=true")
+                    .to("mock:result");
+
+                registrations.forEach(r ->
+                    fromF("jetty:http://%s:%d", r.getAddress().get(), r.getPort().get())
+                        .transform().simple("${in.body} on " + r.getPort().get())
+                );
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/80f746df/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulServiceCallRouteTest.java b/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulServiceCallRouteTest.java
deleted file mode 100644
index c32ec3e..0000000
--- a/components/camel-consul/src/test/java/org/apache/camel/component/consul/cloud/ConsulServiceCallRouteTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.camel.component.consul.cloud;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import com.orbitz.consul.AgentClient;
-import com.orbitz.consul.model.agent.ImmutableRegistration;
-import com.orbitz.consul.model.agent.Registration;
-import org.apache.camel.RoutesBuilder;
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.cloud.ServiceDiscovery;
-import org.apache.camel.component.consul.ConsulConfiguration;
-import org.apache.camel.component.consul.ConsulTestSupport;
-import org.junit.Test;
-
-public class ConsulServiceCallRouteTest extends ConsulTestSupport {
-    private static final String SERVICE_NAME = "http-service";
-    private static final int SERVICE_COUNT = 5;
-    private static final int SERVICE_PORT_BASE = 8080;
-
-    private AgentClient client;
-    private List<Registration> registrations;
-    private List<String> expectedBodies;
-
-    // *************************************************************************
-    // Setup / tear down
-    // *************************************************************************
-
-    @Override
-    protected void doPreSetup() throws Exception {
-        client = getConsul().agentClient();
-
-        registrations = new ArrayList<>(SERVICE_COUNT);
-        expectedBodies = new ArrayList<>(SERVICE_COUNT);
-
-        for (int i = 0; i < SERVICE_COUNT; i++) {
-            Registration r = ImmutableRegistration.builder()
-                .id("service-" + i)
-                .name(SERVICE_NAME)
-                .address("127.0.0.1")
-                .port(SERVICE_PORT_BASE + i)
-                .build();
-
-            client.register(r);
-
-            registrations.add(r);
-            expectedBodies.add("ping on " + r.getPort().get());
-        }
-    }
-
-    @Override
-    public void tearDown() throws Exception {
-        super.tearDown();
-
-
-        registrations.forEach(r -> client.deregister(r.getId()));
-    }
-
-    // *************************************************************************
-    // Test
-    // *************************************************************************
-
-    @Test
-    public void testServiceCall() throws Exception {
-        getMockEndpoint("mock:result").expectedMessageCount(SERVICE_COUNT);
-        getMockEndpoint("mock:result").expectedBodiesReceivedInAnyOrder(expectedBodies);
-
-        registrations.forEach(r -> template.sendBody("direct:start", "ping"));
-
-        assertMockEndpointsSatisfied();
-    }
-
-    // *************************************************************************
-    // Route
-    // *************************************************************************
-
-    @Override
-    protected RoutesBuilder createRouteBuilder() throws Exception {
-        return new RouteBuilder() {
-            @Override
-            public void configure() throws Exception {
-                ConsulConfiguration configuration = new ConsulConfiguration(null);
-                ServiceDiscovery discovery = new ConsulServiceDiscovery(configuration);
-
-                from("direct:start")
-                    .serviceCall()
-                        .name(SERVICE_NAME)
-                        .component("http")
-                        .serviceDiscovery(discovery)
-                        .end()
-                    .to("log:org.apache.camel.component.consul.processor.service?level=INFO&showAll=true&multiline=true")
-                    .to("mock:result");
-
-                registrations.forEach(r ->
-                    fromF("jetty:http://%s:%d", r.getAddress().get(), r.getPort().get())
-                        .transform().simple("${in.body} on " + r.getPort().get())
-                );
-            }
-        };
-    }
-}