You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2016/05/23 08:05:06 UTC

[19/34] camel git commit: CAMEL-9683: Started on camel-ribbon

CAMEL-9683: Started on camel-ribbon


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

Branch: refs/heads/remoteServiceCall
Commit: 9d44cfc92d91c600e1a4c85c857c9f998e46193c
Parents: 28ba0e5
Author: Claus Ibsen <da...@apache.org>
Authored: Mon May 16 11:24:26 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon May 23 09:26:52 2016 +0200

----------------------------------------------------------------------
 .../apache/camel/model/ProcessorDefinition.java | 20 ++++--
 .../ServiceCallConfigurationDefinition.java     | 24 ++++++-
 .../camel/model/ServiceCallDefinition.java      | 32 +++-------
 .../processor/KubernetesProcessorFactory.java   | 28 +++++++--
 .../processor/ServiceCallRouteTest.java         | 17 ++++-
 .../processor/RibbonProcessorFactory.java       | 29 +++++++--
 ...bbonServiceCallStaticServerListStrategy.java |  8 +++
 .../ribbon/processor/RibbonServerListTest.java  |  2 +-
 .../RibbonServiceCallKubernetesRouteTest.java   |  2 +-
 .../RibbonServiceCallRegistryRouteTest.java     | 66 ++++++++++++++++++++
 .../processor/RibbonServiceCallRouteTest.java   | 17 ++---
 11 files changed, 187 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
index 75526c3..469bbe1 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinition.java
@@ -656,15 +656,27 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
     /**
      * Calls the service
      *
+     * @return the builder
+     */
+    @SuppressWarnings("unchecked")
+    public ServiceCallDefinition serviceCall() {
+        ServiceCallDefinition answer = new ServiceCallDefinition();
+        addOutput(answer);
+        return answer;
+    }
+
+    /**
+     * Calls the service
+     *
      * @param name the service name
      * @return the builder
      */
     @SuppressWarnings("unchecked")
-    public ServiceCallDefinition serviceCall(String name) {
+    public Type serviceCall(String name) {
         ServiceCallDefinition answer = new ServiceCallDefinition();
         answer.setName(name);
         addOutput(answer);
-        return answer;
+        return (Type) this;
     }
 
     /**
@@ -672,15 +684,13 @@ public abstract class ProcessorDefinition<Type extends ProcessorDefinition<Type>
      *
      * @param name the service name
      * @param uri  the endpoint uri to use for calling the service
-     * @param configuration the configuration to use
      * @return the builder
      */
     @SuppressWarnings("unchecked")
-    public Type serviceCall(String name, String uri, ServiceCallConfigurationDefinition configuration) {
+    public Type serviceCall(String name, String uri) {
         ServiceCallDefinition answer = new ServiceCallDefinition();
         answer.setName(name);
         answer.setUri(uri);
-        answer.setServiceCallConfiguration(configuration);
         addOutput(answer);
         return (Type) this;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/camel-core/src/main/java/org/apache/camel/model/ServiceCallConfigurationDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ServiceCallConfigurationDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ServiceCallConfigurationDefinition.java
index 34fe14d..9b38565 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ServiceCallConfigurationDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ServiceCallConfigurationDefinition.java
@@ -33,6 +33,8 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
 
     @XmlTransient
     private ServiceCallDefinition parent;
+    @XmlAttribute
+    private String component;
     @XmlAttribute @Metadata(required = "true")
     private String masterUrl;
     @XmlAttribute
@@ -82,6 +84,15 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
     // Getter/Setter
     // -------------------------------------------------------------------------
 
+
+    public String getComponent() {
+        return component;
+    }
+
+    public void setComponent(String component) {
+        this.component = component;
+    }
+
     public String getMasterUrl() {
         return masterUrl;
     }
@@ -238,6 +249,14 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
     // -------------------------------------------------------------------------
 
     /**
+     * Sets the name of the Camel component to use such as ribbon or kubernetes
+     */
+    public ServiceCallConfigurationDefinition component(String component) {
+        setComponent(component);
+        return this;
+    }
+
+    /**
      * Sets the URL to the master
      */
     public ServiceCallConfigurationDefinition masterUrl(String masterUrl) {
@@ -392,8 +411,9 @@ public class ServiceCallConfigurationDefinition extends IdentifiedType {
     /**
      * End of configuration
      */
-    public ServiceCallDefinition end() {
-        return parent;
+    public ProcessorDefinition end() {
+        // end parent as well so we do not have to use 2x end
+        return parent.end();
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/camel-core/src/main/java/org/apache/camel/model/ServiceCallDefinition.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ServiceCallDefinition.java b/camel-core/src/main/java/org/apache/camel/model/ServiceCallDefinition.java
index f381fa0..4cc8cb6 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ServiceCallDefinition.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ServiceCallDefinition.java
@@ -29,6 +29,7 @@ import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.ServiceCallLoadBalancer;
 import org.apache.camel.spi.ServiceCallServerListStrategy;
+import org.apache.camel.util.CamelContextHelper;
 
 @Metadata(label = "eip,routing")
 @XmlRootElement(name = "serviceCall")
@@ -46,8 +47,6 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
     @XmlAttribute @Metadata(required = "true")
     private String name;
     @XmlAttribute
-    private String discovery;
-    @XmlAttribute
     private String serviceCallConfigurationRef;
     @XmlAttribute
     private String loadBalancerRef;
@@ -74,8 +73,14 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
 
     @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {
-        if (discovery != null) {
-            throw new IllegalStateException("Cannot find Camel component on the classpath implementing the discovery provider: " + discovery);
+        String component = serviceCallConfiguration != null ? serviceCallConfiguration.getComponent() : null;
+        if (component == null && serviceCallConfigurationRef != null) {
+            ServiceCallConfigurationDefinition config = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), serviceCallConfigurationRef, ServiceCallConfigurationDefinition.class);
+            component = config.getComponent();
+        }
+
+        if (component != null) {
+            throw new IllegalStateException("Cannot find Camel component on the classpath implementing the discovery provider: " + component);
         } else {
             throw new IllegalStateException("Cannot find Camel component supporting the ServiceCall EIP such as camel-kubernetes or camel-ribbon.");
         }
@@ -117,17 +122,6 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
     }
 
     /**
-     * Sets the discovery provided to use.
-     * <p/>
-     * Use kubernetes to use kubernetes.
-     * Use ribbon to use ribbon.
-     */
-    public ServiceCallDefinition discovery(String discovery) {
-        setDiscovery(discovery);
-        return this;
-    }
-
-    /**
      * Configures the Service Call EIP
      * <p/>
      * Use <tt>end</tt> when configuration is complete, to return back to the Service Call EIP.
@@ -212,14 +206,6 @@ public class ServiceCallDefinition extends NoOutputDefinition<ServiceCallDefinit
         this.pattern = pattern;
     }
 
-    public String getDiscovery() {
-        return discovery;
-    }
-
-    public void setDiscovery(String discovery) {
-        this.discovery = discovery;
-    }
-
     public ServiceCallConfigurationDefinition getServiceCallConfiguration() {
         return serviceCallConfiguration;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-kubernetes/src/main/java/org/apache/camel/component/kubernetes/processor/KubernetesProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-kubernetes/src/main/java/org/apache/camel/component/kubernetes/processor/KubernetesProcessorFactory.java b/components/camel-kubernetes/src/main/java/org/apache/camel/component/kubernetes/processor/KubernetesProcessorFactory.java
index 7ecc6b2..fcf2a65 100644
--- a/components/camel-kubernetes/src/main/java/org/apache/camel/component/kubernetes/processor/KubernetesProcessorFactory.java
+++ b/components/camel-kubernetes/src/main/java/org/apache/camel/component/kubernetes/processor/KubernetesProcessorFactory.java
@@ -18,6 +18,7 @@ package org.apache.camel.component.kubernetes.processor;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Processor;
@@ -49,11 +50,6 @@ public class KubernetesProcessorFactory implements ProcessorFactory {
         if (definition instanceof ServiceCallDefinition) {
             ServiceCallDefinition sc = (ServiceCallDefinition) definition;
 
-            // discovery must either not be set, or if set then must be us
-            if (sc.getDiscovery() != null && !"kubernetes".equals(sc.getDiscovery())) {
-                return null;
-            }
-
             String name = sc.getName();
             String namespace = sc.getNamespace();
             String uri = sc.getUri();
@@ -65,6 +61,28 @@ public class KubernetesProcessorFactory implements ProcessorFactory {
                 configRef = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), sc.getServiceCallConfigurationRef(), ServiceCallConfigurationDefinition.class);
             }
 
+            // if no configuration explicit configured then try to lookup in registry by type and find the best candidate to use
+            if (config == null && configRef == null) {
+                Set<ServiceCallConfigurationDefinition> set = routeContext.getCamelContext().getRegistry().findByType(ServiceCallConfigurationDefinition.class);
+                if (set != null) {
+                    for (ServiceCallConfigurationDefinition candidate : set) {
+                        if (candidate.getComponent() == null || "kubernetes".equals(candidate.getComponent())) {
+                            config = candidate;
+                            break;
+                        }
+                    }
+                }
+            }
+
+            // component must either not be set, or if set then must be us
+            String component = config != null ? config.getComponent() : null;
+            if (component == null && configRef != null) {
+                component = configRef.getComponent();
+            }
+            if (component != null && !"kubernetes".equals(component)) {
+                return null;
+            }
+
             // extract the properties from the configuration from the model
             Map<String, Object> parameters = new HashMap<>();
             if (configRef != null) {

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-kubernetes/src/test/java/org/apache/camel/component/kubernetes/processor/ServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-kubernetes/src/test/java/org/apache/camel/component/kubernetes/processor/ServiceCallRouteTest.java b/components/camel-kubernetes/src/test/java/org/apache/camel/component/kubernetes/processor/ServiceCallRouteTest.java
index 054c2f6..0df2937 100644
--- a/components/camel-kubernetes/src/test/java/org/apache/camel/component/kubernetes/processor/ServiceCallRouteTest.java
+++ b/components/camel-kubernetes/src/test/java/org/apache/camel/component/kubernetes/processor/ServiceCallRouteTest.java
@@ -18,6 +18,7 @@ package org.apache.camel.component.kubernetes.processor;
 
 import org.apache.camel.RoutesBuilder;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.JndiRegistry;
 import org.apache.camel.model.ServiceCallConfigurationDefinition;
 import org.apache.camel.test.junit4.CamelTestSupport;
 import org.junit.Ignore;
@@ -26,6 +27,14 @@ import org.junit.Test;
 @Ignore("Manual test")
 public class ServiceCallRouteTest extends CamelTestSupport {
 
+    private JndiRegistry registry;
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        registry = super.createRegistry();
+        return registry;
+    }
+
     @Test
     public void testServiceCall() throws Exception {
         getMockEndpoint("mock:result").expectedMessageCount(1);
@@ -48,10 +57,12 @@ public class ServiceCallRouteTest extends CamelTestSupport {
                 // lets use the built-in round robin (random is default)
                 config.setLoadBalancerRef("roundrobin");
 
+                // add the config to the registry so service call can use it
+                registry.bind("myConfig", config);
+
                 from("direct:start")
-//                    .serviceCall("cdi-camel-jetty", null, config)
-                    .to("http:{{service:cdi-camel-jetty}}")
-                    .serviceCall("cdi-camel-jetty", null, config)
+                    .serviceCall("cdi-camel-jetty")
+                    .serviceCall("cdi-camel-jetty")
                     .to("mock:result");
             }
         };

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonProcessorFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonProcessorFactory.java b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonProcessorFactory.java
index 15ebef5..5a3f685 100644
--- a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonProcessorFactory.java
+++ b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonProcessorFactory.java
@@ -18,6 +18,7 @@ package org.apache.camel.component.ribbon.processor;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import com.netflix.loadbalancer.IRule;
 import org.apache.camel.ExchangePattern;
@@ -49,11 +50,6 @@ public class RibbonProcessorFactory implements ProcessorFactory {
         if (definition instanceof ServiceCallDefinition) {
             ServiceCallDefinition sc = (ServiceCallDefinition) definition;
 
-            // discovery must either not be set, or if set then must be us
-            if (sc.getDiscovery() != null && !"ribbon".equals(sc.getDiscovery())) {
-                return null;
-            }
-
             String name = sc.getName();
             String namespace = sc.getNamespace();
             String uri = sc.getUri();
@@ -65,6 +61,19 @@ public class RibbonProcessorFactory implements ProcessorFactory {
                 configRef = CamelContextHelper.mandatoryLookup(routeContext.getCamelContext(), sc.getServiceCallConfigurationRef(), ServiceCallConfigurationDefinition.class);
             }
 
+            // if no configuration explicit configured then try to lookup in registry by type and find the best candidate to use
+            if (config == null && configRef == null) {
+                Set<ServiceCallConfigurationDefinition> set = routeContext.getCamelContext().getRegistry().findByType(ServiceCallConfigurationDefinition.class);
+                if (set != null) {
+                    for (ServiceCallConfigurationDefinition candidate : set) {
+                        if (candidate.getComponent() == null || "kubernetes".equals(candidate.getComponent())) {
+                            config = candidate;
+                            break;
+                        }
+                    }
+                }
+            }
+
             // extract the properties from the configuration from the model
             Map<String, Object> parameters = new HashMap<>();
             if (configRef != null) {
@@ -73,6 +82,16 @@ public class RibbonProcessorFactory implements ProcessorFactory {
             if (config != null) {
                 IntrospectionSupport.getProperties(config, parameters, null);
             }
+
+            // component must either not be set, or if set then must be us
+            String component = config != null ? config.getComponent() : null;
+            if (component == null && configRef != null) {
+                component = configRef.getComponent();
+            }
+            if (component != null && !"ribbon".equals(component)) {
+                return null;
+            }
+
             // and set them on the kubernetes configuration class
             RibbonConfiguration rc = new RibbonConfiguration();
             IntrospectionSupport.setProperties(rc, parameters);

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallStaticServerListStrategy.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallStaticServerListStrategy.java b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallStaticServerListStrategy.java
index e315fe9..5891360 100644
--- a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallStaticServerListStrategy.java
+++ b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallStaticServerListStrategy.java
@@ -36,6 +36,14 @@ public class RibbonServiceCallStaticServerListStrategy extends AbstractServerLis
         this.servers.addAll(servers);
     }
 
+    public void addServer(RibbonServer server) {
+        servers.add(server);
+    }
+
+    public void addServer(String host, int port) {
+        servers.add(new RibbonServer(host, port));
+    }
+
     @Override
     public void initWithNiwsConfig(IClientConfig clientConfig) {
         this.clientConfig = clientConfig;

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServerListTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServerListTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServerListTest.java
index 38ea4d1..0127d9d 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServerListTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServerListTest.java
@@ -31,7 +31,7 @@ public class RibbonServerListTest extends TestCase {
 
     @Test
     public void testFixedServerList() throws Exception {
-        List<RibbonServer> servers = new ArrayList();
+        List<RibbonServer> servers = new ArrayList<>();
         servers.add(new RibbonServer("localhost", 9090));
         servers.add(new RibbonServer("localhost", 9091));
 

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallKubernetesRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallKubernetesRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallKubernetesRouteTest.java
index db67030..2a945c3 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallKubernetesRouteTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallKubernetesRouteTest.java
@@ -47,7 +47,7 @@ public class RibbonServiceCallKubernetesRouteTest extends CamelTestSupport {
                 config.setNamespace("default");
 
                 from("direct:start")
-                        .serviceCall("cdi-camel-jetty", null, config)
+                        .serviceCall("cdi-camel-jetty")
                         .to("mock:result");
             }
         };

http://git-wip-us.apache.org/repos/asf/camel/blob/9d44cfc9/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRegistryRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRegistryRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRegistryRouteTest.java
new file mode 100644
index 0000000..5d33e61
--- /dev/null
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRegistryRouteTest.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.processor;
+
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.model.ServiceCallConfigurationDefinition;
+
+public class RibbonServiceCallRegistryRouteTest extends RibbonServiceCallRouteTest {
+
+    private JndiRegistry registry;
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        registry = super.createRegistry();
+        return registry;
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // setup a static ribbon server list with these 2 servers to start with
+                RibbonServiceCallStaticServerListStrategy servers = new RibbonServiceCallStaticServerListStrategy();
+                servers.addServer("localhost", 9090);
+                servers.addServer("localhost", 9091);
+
+                // configure camel service call
+                ServiceCallConfigurationDefinition config = new ServiceCallConfigurationDefinition();
+                config.setServerListStrategy(servers);
+
+                // add the config to the registry so service call can use it
+                registry.bind("myConfig", config);
+
+                from("direct:start")
+                        .serviceCall("myService")
+                        .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/9d44cfc9/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRouteTest.java b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRouteTest.java
index 52b631b..a57bced 100644
--- a/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRouteTest.java
+++ b/components/camel-ribbon/src/test/java/org/apache/camel/component/ribbon/processor/RibbonServiceCallRouteTest.java
@@ -16,12 +16,8 @@
  */
 package org.apache.camel.component.ribbon.processor;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.camel.RoutesBuilder;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.model.ServiceCallConfigurationDefinition;
 import org.apache.camel.test.junit4.CamelTestSupport;
 import org.junit.Test;
 
@@ -47,17 +43,12 @@ public class RibbonServiceCallRouteTest extends CamelTestSupport {
             @Override
             public void configure() throws Exception {
                 // setup a static ribbon server list with these 2 servers to start with
-                List<RibbonServer> servers = new ArrayList<>();
-                servers.add(new RibbonServer("localhost", 9090));
-                servers.add(new RibbonServer("localhost", 9091));
-                RibbonServiceCallStaticServerListStrategy list = new RibbonServiceCallStaticServerListStrategy(servers);
-
-                // configure camel service call
-                ServiceCallConfigurationDefinition config = new ServiceCallConfigurationDefinition();
-                config.setServerListStrategy(list);
+                RibbonServiceCallStaticServerListStrategy servers = new RibbonServiceCallStaticServerListStrategy();
+                servers.addServer("localhost", 9090);
+                servers.addServer("localhost", 9091);
 
                 from("direct:start")
-                        .serviceCall("myService", null, config)
+                        .serviceCall().name("myService").serviceCallConfiguration().serverListStrategy(servers).end()
                         .to("mock:result");
 
                 from("jetty:http://localhost:9090")