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 2016/12/12 12:44:12 UTC

camel git commit: CAMEL-10027: ServiceCall EIP : Support additional attributes in ServiceCallServer

Repository: camel
Updated Branches:
  refs/heads/master a5771f5a8 -> 31388a560


CAMEL-10027: ServiceCall EIP : Support additional attributes in ServiceCallServer


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

Branch: refs/heads/master
Commit: 31388a56036846034c311b666911c8f09a229458
Parents: a5771f5
Author: lburgazzoli <lb...@gmail.com>
Authored: Mon Dec 12 13:21:17 2016 +0100
Committer: lburgazzoli <lb...@gmail.com>
Committed: Mon Dec 12 13:21:17 2016 +0100

----------------------------------------------------------------------
 .../impl/remote/DefaultServiceCallServer.java   | 17 ++++
 .../org/apache/camel/spi/ServiceCallServer.java |  8 +-
 .../org/apache/camel/util/ObjectHelper.java     | 13 +++
 .../apache/camel/util/function/Bindings.java    | 35 ++++++++
 .../ConsulServiceCallServerListStrategy.java    | 24 ++++-
 .../remote/ConsulServiceCallRouteTest.java      |  2 +-
 ...nsulServiceCallServerListStrategiesTest.java | 92 ++++++++++++++++++++
 .../processor/remote/DnsServiceCallServer.java  | 18 +++-
 .../remote/DnsServiceLookupFactory.java         | 17 ++--
 .../DnsServiceCallServerListStrategiesTest.java | 47 ++++++++++
 .../processor/remote/EtcdServiceCallServer.java |  9 +-
 .../remote/EtcdServiceCallRouteTest.java        |  2 +
 .../EtcdServiceCallServerListStrategyTest.java  | 26 +++++-
 .../ribbon/processor/RibbonServer.java          | 28 +++++-
 14 files changed, 309 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/camel-core/src/main/java/org/apache/camel/impl/remote/DefaultServiceCallServer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/remote/DefaultServiceCallServer.java b/camel-core/src/main/java/org/apache/camel/impl/remote/DefaultServiceCallServer.java
index e42afc7..30aac8d 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/remote/DefaultServiceCallServer.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/remote/DefaultServiceCallServer.java
@@ -16,16 +16,28 @@
  */
 package org.apache.camel.impl.remote;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.camel.spi.ServiceCallServer;
 
 public class DefaultServiceCallServer implements ServiceCallServer {
 
     private final String ip;
     private final int port;
+    private final Map<String, String> meta;
 
     public DefaultServiceCallServer(String ip, int port) {
         this.ip = ip;
         this.port = port;
+        this.meta = Collections.emptyMap();
+    }
+
+    public DefaultServiceCallServer(String ip, int port, Map<String, String> meta) {
+        this.ip = ip;
+        this.port = port;
+        this.meta = Collections.unmodifiableMap(new HashMap<>(meta));
     }
 
     @Override
@@ -39,6 +51,11 @@ public class DefaultServiceCallServer implements ServiceCallServer {
     }
 
     @Override
+    public Map<String, String> getMetadata() {
+        return this.meta;
+    }
+
+    @Override
     public String toString() {
         return "DefaultServiceCallServer[" + ip + ":" + port + "]";
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/camel-core/src/main/java/org/apache/camel/spi/ServiceCallServer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/ServiceCallServer.java b/camel-core/src/main/java/org/apache/camel/spi/ServiceCallServer.java
index 1579599..bf79482 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/ServiceCallServer.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/ServiceCallServer.java
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.spi;
 
+import java.util.Map;
+
 /**
  * Represents a server that host a service for the Service Call EIP.
  *
@@ -23,7 +25,6 @@ package org.apache.camel.spi;
  * @see ServiceCallServerListStrategy
  */
 public interface ServiceCallServer {
-
     /**
      * Gets the IP or hostname of the server hosting the service
      */
@@ -33,4 +34,9 @@ public interface ServiceCallServer {
      * Gets the port number of the server hosting the service
      */
     int getPort();
+
+    /**
+     * Gets a key/value metadata associated with the service
+     */
+    Map<String, String> getMetadata();
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java b/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
index dcfbf87..2a9b511 100644
--- a/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
+++ b/camel-core/src/main/java/org/apache/camel/util/ObjectHelper.java
@@ -46,6 +46,7 @@ import java.util.Optional;
 import java.util.Properties;
 import java.util.Scanner;
 import java.util.concurrent.Callable;
+import java.util.function.Consumer;
 import java.util.function.Function;
 
 import org.w3c.dom.Node;
@@ -391,6 +392,18 @@ public final class ObjectHelper {
     }
 
     /**
+     * Tests whether the value is <b>not</b> <tt>null</tt> or an empty string.
+     *
+     * @param value  the value, if its a String it will be tested for text length as well
+     * @param consumer  the consumer, the operation to be executed against value if not empty
+     */
+    public static <T> void ifNotEmpty(T value, Consumer<T> consumer) {
+        if (isNotEmpty(value)) {
+            consumer.accept(value);
+        }
+    }
+
+    /**
      * @deprecated use
      *             {@link StringHelper#splitOnCharacter(String, String, int)} instead
      */

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/camel-core/src/main/java/org/apache/camel/util/function/Bindings.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/util/function/Bindings.java b/camel-core/src/main/java/org/apache/camel/util/function/Bindings.java
new file mode 100644
index 0000000..e8adb99
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/util/function/Bindings.java
@@ -0,0 +1,35 @@
+/**
+ * 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.util.function;
+
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+public final class Bindings {
+    private Bindings() {
+    }
+
+    public static <T1, T2> Consumer<T2> bind(T1 v1, BiConsumer<T1, T2> consumer) {
+        return v2 -> consumer.accept(v1, v2);
+    }
+
+    public static <T1, T2, R> Function<T2, R> bind(T1 v1, BiFunction<T1, T2, R> function) {
+        return v2 -> function.apply(v1, v2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-consul/src/main/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategy.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/main/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategy.java b/components/camel-consul/src/main/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategy.java
index 1db4bc4..116b2fc 100644
--- a/components/camel-consul/src/main/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategy.java
+++ b/components/camel-consul/src/main/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategy.java
@@ -16,7 +16,9 @@
  */
 package org.apache.camel.component.consul.processor.remote;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import com.orbitz.consul.CatalogClient;
 import com.orbitz.consul.Consul;
@@ -32,6 +34,8 @@ import org.apache.camel.impl.remote.DefaultServiceCallServerListStrategy;
 import org.apache.camel.spi.ServiceCallServer;
 import org.apache.camel.util.ObjectHelper;
 
+import static org.apache.camel.util.ObjectHelper.ifNotEmpty;
+
 
 abstract class ConsulServiceCallServerListStrategy extends DefaultServiceCallServerListStrategy<ServiceCallServer> {
     private final Consul client;
@@ -98,9 +102,27 @@ abstract class ConsulServiceCallServerListStrategy extends DefaultServiceCallSer
     }
 
     protected ServiceCallServer newServer(CatalogService service) {
+        Map<String, String> meta = new HashMap<>();
+        ifNotEmpty(service.getServiceId(), val -> meta.put("service_id", val));
+        ifNotEmpty(service.getNode(), val -> meta.put("node", val));
+        ifNotEmpty(service.getServiceName(), val -> meta.put("service_name", val));
+
+        List<String> tags = service.getServiceTags();
+        if (tags != null) {
+            for (String tag : service.getServiceTags()) {
+                String[] items = tag.split("=");
+                if (items.length == 1) {
+                    meta.put(items[0], items[0]);
+                } else if (items.length == 2) {
+                    meta.put(items[0], items[1]);
+                }
+            }
+        }
+
         return new DefaultServiceCallServer(
             service.getServiceAddress(),
-            service.getServicePort()
+            service.getServicePort(),
+            meta
         );
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallRouteTest.java b/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallRouteTest.java
index 7eb6b40..a76580b 100644
--- a/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallRouteTest.java
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallRouteTest.java
@@ -28,7 +28,7 @@ import org.apache.camel.component.consul.ConsulTestSupport;
 import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore
+@Ignore("A consul server is needed for this test ")
 public class ConsulServiceCallRouteTest extends ConsulTestSupport {
     private static final String SERVICE_NAME = "http-service";
     private static final int SERVICE_COUNT = 5;

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategiesTest.java
----------------------------------------------------------------------
diff --git a/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategiesTest.java b/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategiesTest.java
new file mode 100644
index 0000000..65738e0
--- /dev/null
+++ b/components/camel-consul/src/test/java/org/apache/camel/component/consul/processor/remote/ConsulServiceCallServerListStrategiesTest.java
@@ -0,0 +1,92 @@
+/**
+ * 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.processor.remote;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.orbitz.consul.AgentClient;
+import com.orbitz.consul.Consul;
+import com.orbitz.consul.model.agent.ImmutableRegistration;
+import com.orbitz.consul.model.agent.Registration;
+import org.apache.camel.component.consul.ConsulConfiguration;
+import org.apache.camel.spi.ServiceCallServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@Ignore("A consul server is needed for this test ")
+public class ConsulServiceCallServerListStrategiesTest {
+    private AgentClient client;
+    private List<Registration> registrations;
+
+    @Before
+    public void setUp() throws Exception {
+        client = Consul.builder().build().agentClient();
+        registrations = new ArrayList<>(3);
+
+        for (int i = 0; i < 3; i++) {
+            Registration r = ImmutableRegistration.builder()
+                .id("service-" + i)
+                .name("my-service")
+                .address("127.0.0.1")
+                .addTags("a-tag")
+                .addTags("key1=value1")
+                .addTags("key2=value2")
+                .port(9000 + i)
+                .build();
+
+            client.register(r);
+            registrations.add(r);
+        }
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        registrations.forEach(r -> client.deregister(r.getId()));
+    }
+
+    // *************************************************************************
+    // Test
+    // *************************************************************************
+
+    @Test
+    public void testOnDemand() throws Exception {
+        ConsulConfiguration configuration = new ConsulConfiguration(null);
+        ConsulServiceCallServerListStrategy strategy = ConsulServiceCallServerListStrategies.onDemand(configuration);
+
+        List<ServiceCallServer> servers = strategy.getUpdatedListOfServers("my-service");
+        assertNotNull(servers);
+        assertEquals(3, servers.size());
+
+        for (ServiceCallServer server : servers) {
+            assertFalse(server.getMetadata().isEmpty());
+            assertTrue(server.getMetadata().containsKey("service_name"));
+            assertTrue(server.getMetadata().containsKey("service_id"));
+            assertTrue(server.getMetadata().containsKey("a-tag"));
+            assertTrue(server.getMetadata().containsKey("key1"));
+            assertTrue(server.getMetadata().containsKey("key2"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServer.java
----------------------------------------------------------------------
diff --git a/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServer.java b/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServer.java
index 11fe753..77b7082 100644
--- a/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServer.java
+++ b/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServer.java
@@ -17,15 +17,23 @@
 package org.apache.camel.component.dns.processor.remote;
 
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.camel.impl.remote.DefaultServiceCallServer;
 import org.xbill.DNS.SRVRecord;
 
+import static org.apache.camel.util.ObjectHelper.ifNotEmpty;
+
 public class DnsServiceCallServer extends DefaultServiceCallServer {
     public static final Comparator<SRVRecord> COMPARATOR = comparator();
 
     public DnsServiceCallServer(SRVRecord record) {
-        super(record.getTarget().toString(true), record.getPort());
+        super(
+            record.getTarget().toString(true),
+            record.getPort(),
+            getRecordMetaData(record)
+        );
     }
 
     public static Comparator<SRVRecord> comparator() {
@@ -34,4 +42,12 @@ public class DnsServiceCallServer extends DefaultServiceCallServer {
 
         return byPriority.thenComparing(byWeight);
     }
+
+    public static Map<String, String> getRecordMetaData(SRVRecord record) {
+        Map<String, String> meta = new HashMap<>();
+        ifNotEmpty(record.getPriority(), val -> meta.put("priority", Integer.toString(val)));
+        ifNotEmpty(record.getWeight(), val -> meta.put("weight", Integer.toString(val)));
+
+        return meta;
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceLookupFactory.java
----------------------------------------------------------------------
diff --git a/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceLookupFactory.java b/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceLookupFactory.java
index 49d6b21..ce63c96 100644
--- a/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceLookupFactory.java
+++ b/components/camel-dns/src/main/java/org/apache/camel/component/dns/processor/remote/DnsServiceLookupFactory.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.camel.component.dns.processor.remote;
 
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Function;
 
@@ -28,23 +28,16 @@ import org.xbill.DNS.Type;
 
 public class DnsServiceLookupFactory implements Function<String, Lookup> {
     private final DnsConfiguration configuration;
-    private final Map<String, Lookup> cache;
+    private final ConcurrentHashMap<String, Lookup> cache;
 
-    public  DnsServiceLookupFactory(DnsConfiguration configuration) {
+    public DnsServiceLookupFactory(DnsConfiguration configuration) {
         this.configuration = configuration;
-        cache = new ConcurrentHashMap<>();
+        this.cache = new ConcurrentHashMap<>();
     }
 
     @Override
     public Lookup apply(String name) {
-        Lookup lookup = cache.get(name);
-        if (lookup == null) {
-            synchronized (cache) {
-                lookup = cache.computeIfAbsent(name, this::createLookup);
-            }
-        }
-
-        return lookup;
+        return cache.computeIfAbsent(name, this::createLookup);
     }
 
     private Lookup createLookup(String name) {

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-dns/src/test/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServerListStrategiesTest.java
----------------------------------------------------------------------
diff --git a/components/camel-dns/src/test/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServerListStrategiesTest.java b/components/camel-dns/src/test/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServerListStrategiesTest.java
new file mode 100644
index 0000000..992f7ce
--- /dev/null
+++ b/components/camel-dns/src/test/java/org/apache/camel/component/dns/processor/remote/DnsServiceCallServerListStrategiesTest.java
@@ -0,0 +1,47 @@
+/**
+ * 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.dns.processor.remote;
+
+import java.util.List;
+
+import org.apache.camel.component.dns.DnsConfiguration;
+import org.apache.camel.spi.ServiceCallServer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+public class DnsServiceCallServerListStrategiesTest {
+    @Test
+    public void testOnDemand() throws Exception {
+        DnsConfiguration configuration = new DnsConfiguration();
+        DnsServiceCallServerListStrategy strategy = DnsServiceCallServerListStrategies.onDemand(configuration);
+
+        configuration.setDomain("gmail.com");
+        configuration.setProto("_tcp");
+
+        List<ServiceCallServer> servers = strategy.getUpdatedListOfServers("_xmpp-server");
+        assertNotNull(servers);
+        assertFalse(servers.isEmpty());
+
+        for (ServiceCallServer server : servers) {
+            assertFalse(server.getMetadata().isEmpty());
+            assertNotNull(server.getMetadata().get("dns.priority"));
+            assertNotNull(server.getMetadata().get("dns.weight"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-etcd/src/main/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServer.java
----------------------------------------------------------------------
diff --git a/components/camel-etcd/src/main/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServer.java b/components/camel-etcd/src/main/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServer.java
index d5c25d9..648b544 100644
--- a/components/camel-etcd/src/main/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServer.java
+++ b/components/camel-etcd/src/main/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServer.java
@@ -16,7 +16,6 @@
  */
 package org.apache.camel.component.etcd.processor.remote;
 
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.Map;
 
@@ -28,7 +27,6 @@ public class EtcdServiceCallServer extends DefaultServiceCallServer {
     public static final Comparator<EtcdServiceCallServer> COMPARATOR = comparator();
 
     private final String name;
-    private final Map<String, String> tags;
 
     @JsonCreator
     public EtcdServiceCallServer(
@@ -36,20 +34,15 @@ public class EtcdServiceCallServer extends DefaultServiceCallServer {
         @JsonProperty("address") final String address,
         @JsonProperty("port") final Integer port,
         @JsonProperty("tags") final Map<String, String> tags) {
-        super(address, port);
+        super(address, port, tags);
 
         this.name = name;
-        this.tags = Collections.unmodifiableMap(tags != null ? tags : Collections.EMPTY_MAP);
     }
 
     public String getName() {
         return name;
     }
 
-    public Map<String, String> getTags() {
-        return tags;
-    }
-
     public static Comparator<EtcdServiceCallServer> comparator() {
         Comparator<EtcdServiceCallServer> byAddress = (e1, e2) -> e2.getIp().compareTo(e1.getIp());
         Comparator<EtcdServiceCallServer> byPort = (e1, e2) -> Integer.compare(e2.getPort(), e1.getPort());

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallRouteTest.java
----------------------------------------------------------------------
diff --git a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallRouteTest.java b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallRouteTest.java
index 00d75d3..7987ec4 100644
--- a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallRouteTest.java
+++ b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallRouteTest.java
@@ -29,8 +29,10 @@ import org.apache.camel.RoutesBuilder;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.etcd.EtcdHelper;
 import org.apache.camel.component.etcd.EtcdTestSupport;
+import org.junit.Ignore;
 import org.junit.Test;
 
+@Ignore("An etcd server is needed for this test ")
 public class EtcdServiceCallRouteTest extends EtcdTestSupport {
     private static final ObjectMapper MAPPER = EtcdHelper.createObjectMapper();
     private static final String SERVICE_NAME = "http-service";

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServerListStrategyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServerListStrategyTest.java b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServerListStrategyTest.java
index 55423be..837f057 100644
--- a/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServerListStrategyTest.java
+++ b/components/camel-etcd/src/test/java/org/apache/camel/component/etcd/processor/remote/EtcdServiceCallServerListStrategyTest.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.camel.component.etcd.processor.remote;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -30,7 +30,7 @@ import org.apache.camel.component.etcd.EtcdTestSupport;
 import org.junit.Ignore;
 import org.junit.Test;
 
-@Ignore
+@Ignore("An etcd server is needed for this test ")
 public class EtcdServiceCallServerListStrategyTest extends EtcdTestSupport {
     private static final ObjectMapper MAPPER = EtcdHelper.createObjectMapper();
     private static final EtcdConfiguration CONFIGURATION = new EtcdConfiguration(null);
@@ -71,8 +71,21 @@ public class EtcdServiceCallServerListStrategyTest extends EtcdTestSupport {
         EtcdServiceCallServerListStrategy strategy = EtcdServiceCallServerListStrategies.onDemand(CONFIGURATION);
         strategy.start();
 
-        assertEquals(3, strategy.getUpdatedListOfServers("serviceType-1").size());
-        assertEquals(2, strategy.getUpdatedListOfServers("serviceType-2").size());
+        List<EtcdServiceCallServer> type1 = strategy.getUpdatedListOfServers("serviceType-1");
+        assertEquals(3, type1.size());
+        for (EtcdServiceCallServer server : type1) {
+            assertNotNull(server.getMetadata());
+            assertTrue(server.getMetadata().containsKey("service_name"));
+            assertTrue(server.getMetadata().containsKey("port_delta"));
+        }
+
+        List<EtcdServiceCallServer> type2 = strategy.getUpdatedListOfServers("serviceType-2");
+        assertEquals(2, type2.size());
+        for (EtcdServiceCallServer server : type2) {
+            assertNotNull(server.getMetadata());
+            assertTrue(server.getMetadata().containsKey("service_name"));
+            assertTrue(server.getMetadata().containsKey("port_delta"));
+        }
 
         strategy.stop();
     }
@@ -100,10 +113,15 @@ public class EtcdServiceCallServerListStrategyTest extends EtcdTestSupport {
     private void addServer(EtcdClient client, String name) throws Exception {
         int port = PORT.incrementAndGet();
 
+        Map<String, String> tags = new HashMap<>();
+        tags.put("service_name", name);
+        tags.put("port_delta", Integer.toString(port));
+
         Map<String, Object> server = new HashMap<>();
         server.put("name", name);
         server.put("address", "127.0.0.1");
         server.put("port", 8000 + port);
+        server.put("tags", tags);
 
         client.put(CONFIGURATION.getServicePath() + "service-" + port, MAPPER.writeValueAsString(server)).send().get();
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/31388a56/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServer.java
----------------------------------------------------------------------
diff --git a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServer.java b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServer.java
index d4e5bbf..d30d656 100644
--- a/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServer.java
+++ b/components/camel-ribbon/src/main/java/org/apache/camel/component/ribbon/processor/RibbonServer.java
@@ -16,9 +16,15 @@
  */
 package org.apache.camel.component.ribbon.processor;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import com.netflix.loadbalancer.Server;
 import org.apache.camel.spi.ServiceCallServer;
 
+import static org.apache.camel.util.ObjectHelper.ifNotEmpty;
+
 public class RibbonServer extends Server implements ServiceCallServer {
 
     public RibbonServer(String host, int port) {
@@ -27,7 +33,27 @@ public class RibbonServer extends Server implements ServiceCallServer {
 
     @Override
     public String getIp() {
-        return getHost();
+        return super.getHost();
+    }
+
+    @Override
+    public int getPort() {
+        return super.getPort();
     }
 
+    @Override
+    public Map<String, String> getMetadata() {
+        Map<String, String> meta = new HashMap<>();
+        ifNotEmpty(super.getId(), val -> meta.put("id", val));
+        ifNotEmpty(super.getZone(), val -> meta.put("zone", val));
+
+        if (super.getMetaInfo() != null) {
+            ifNotEmpty(super.getMetaInfo().getAppName(), val -> meta.put("app_name", val));
+            ifNotEmpty(super.getMetaInfo().getServiceIdForDiscovery(),  val -> meta.put("discovery_id", val));
+            ifNotEmpty(super.getMetaInfo().getInstanceId(),  val -> meta.put("instance_id", val));
+            ifNotEmpty(super.getMetaInfo().getServerGroup(), val -> meta.put("server_group", val));
+        }
+
+        return Collections.unmodifiableMap(meta);
+    }
 }