You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ti...@apache.org on 2023/08/13 15:39:22 UTC

[curator] branch master updated: CURATOR-457: Custom service discovery path (#473)

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

tison pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/curator.git


The following commit(s) were added to refs/heads/master by this push:
     new d14cb464 CURATOR-457: Custom service discovery path (#473)
d14cb464 is described below

commit d14cb464cbd4e8e0e1bb51daba76e84eb2e346db
Author: Kezhu Wang <ke...@apache.org>
AuthorDate: Sun Aug 13 23:39:17 2023 +0800

    CURATOR-457: Custom service discovery path (#473)
    
    Signed-off-by: tison <wa...@gmail.com>
    Co-authored-by: alexm <al...@gmail.com>
    Co-authored-by: tison <wa...@gmail.com>
---
 curator-x-discovery/pom.xml                        |  6 +++
 .../x/discovery/DiscoveryPathConstructor.java      | 40 +++++++++++++++++
 .../x/discovery/ServiceDiscoveryBuilder.java       | 21 +++++++--
 .../details/DiscoveryPathConstructorImpl.java      | 48 ++++++++++++++++++++
 .../x/discovery/details/ServiceDiscoveryImpl.java  | 45 ++++++++++++-------
 .../details/DiscoveryPathConstructorImplTest.java  | 52 ++++++++++++++++++++++
 .../x/discovery/details/TestServiceDiscovery.java  | 44 ++++++++++++++++++
 7 files changed, 237 insertions(+), 19 deletions(-)

diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index c27b648a..229fce2f 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -81,5 +81,11 @@
             <artifactId>commons-math</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.assertj</groupId>
+            <artifactId>assertj-core</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/DiscoveryPathConstructor.java b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/DiscoveryPathConstructor.java
new file mode 100644
index 00000000..3c865dbd
--- /dev/null
+++ b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/DiscoveryPathConstructor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.curator.x.discovery;
+
+/**
+ * Constructs ZooKeeper paths to services for service discovering.
+ */
+public interface DiscoveryPathConstructor {
+    /**
+     * Return the parent path where all service names registered.
+     *
+     * @return the base path of all services
+     */
+    String getBasePath();
+
+    /**
+     * Return the parent path where all instances of the service registered.
+     *
+     * @param serviceName service name
+     * @return  path to service instances
+     */
+    String getPathForInstances(String serviceName);
+}
diff --git a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/ServiceDiscoveryBuilder.java b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/ServiceDiscoveryBuilder.java
index a5159a65..c103847e 100644
--- a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/ServiceDiscoveryBuilder.java
+++ b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/ServiceDiscoveryBuilder.java
@@ -20,13 +20,14 @@
 package org.apache.curator.x.discovery;
 
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.x.discovery.details.DiscoveryPathConstructorImpl;
 import org.apache.curator.x.discovery.details.InstanceSerializer;
 import org.apache.curator.x.discovery.details.JsonInstanceSerializer;
 import org.apache.curator.x.discovery.details.ServiceDiscoveryImpl;
 
 public class ServiceDiscoveryBuilder<T> {
     private CuratorFramework client;
-    private String basePath;
+    private DiscoveryPathConstructor pathConstructor;
     private InstanceSerializer<T> serializer;
     private ServiceInstance<T> thisInstance;
     private Class<T> payloadClass;
@@ -53,7 +54,7 @@ public class ServiceDiscoveryBuilder<T> {
         if (serializer == null) {
             serializer(new JsonInstanceSerializer<T>(payloadClass));
         }
-        return new ServiceDiscoveryImpl<T>(client, basePath, serializer, thisInstance, watchInstances);
+        return new ServiceDiscoveryImpl<T>(client, pathConstructor, serializer, thisInstance, watchInstances);
     }
 
     /**
@@ -68,13 +69,25 @@ public class ServiceDiscoveryBuilder<T> {
     }
 
     /**
-     * Required - set the base path to store in ZK
+     * Required - set the base path to store in ZK, see {@link #pathConstructor(DiscoveryPathConstructor)}
+     * for alternative
      *
      * @param basePath base path
      * @return this
      */
     public ServiceDiscoveryBuilder<T> basePath(String basePath) {
-        this.basePath = basePath;
+        this.pathConstructor = new DiscoveryPathConstructorImpl(basePath);
+        return this;
+    }
+
+    /**
+     * Required - shape the service tree in ZK, see {@link #basePath(String)} for alternative
+     *
+     * @param pathConstructor custom service tree
+     * @return this
+     */
+    public ServiceDiscoveryBuilder<T> pathConstructor(DiscoveryPathConstructor pathConstructor) {
+        this.pathConstructor = pathConstructor;
         return this;
     }
 
diff --git a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImpl.java b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImpl.java
new file mode 100644
index 00000000..3f999cce
--- /dev/null
+++ b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImpl.java
@@ -0,0 +1,48 @@
+/*
+ * 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.curator.x.discovery.details;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.utils.PathUtils;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.curator.x.discovery.DiscoveryPathConstructor;
+
+/**
+ * A standard constructor, it uses standard path constructing strategy by applying name to the base path.
+ */
+public class DiscoveryPathConstructorImpl implements DiscoveryPathConstructor {
+    private final String basePath;
+
+    public DiscoveryPathConstructorImpl(String basePath) {
+        Preconditions.checkArgument(basePath != null, "basePath cannot be null");
+        PathUtils.validatePath(basePath);
+        this.basePath = basePath;
+    }
+
+    @Override
+    public String getBasePath() {
+        return basePath;
+    }
+
+    @Override
+    public String getPathForInstances(String serviceName) {
+        return ZKPaths.makePath(basePath, serviceName);
+    }
+}
diff --git a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/ServiceDiscoveryImpl.java b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/ServiceDiscoveryImpl.java
index c711e443..51aa94bc 100644
--- a/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/ServiceDiscoveryImpl.java
+++ b/curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/ServiceDiscoveryImpl.java
@@ -23,11 +23,11 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.CuratorCache;
@@ -39,6 +39,7 @@ import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ExceptionAccumulator;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZKPaths;
+import org.apache.curator.x.discovery.DiscoveryPathConstructor;
 import org.apache.curator.x.discovery.ServiceCache;
 import org.apache.curator.x.discovery.ServiceCacheBuilder;
 import org.apache.curator.x.discovery.ServiceDiscovery;
@@ -59,13 +60,11 @@ import org.slf4j.LoggerFactory;
 public class ServiceDiscoveryImpl<T> implements ServiceDiscovery<T> {
     private final Logger log = LoggerFactory.getLogger(getClass());
     private final CuratorFramework client;
-    private final String basePath;
+    private final DiscoveryPathConstructor pathConstructor;
     private final InstanceSerializer<T> serializer;
-    private final ConcurrentMap<String, Entry<T>> services = Maps.newConcurrentMap();
-    private final Collection<ServiceCache<T>> caches =
-            Sets.newSetFromMap(Maps.<ServiceCache<T>, Boolean>newConcurrentMap());
-    private final Collection<ServiceProvider<T>> providers =
-            Sets.newSetFromMap(Maps.<ServiceProvider<T>, Boolean>newConcurrentMap());
+    private final ConcurrentMap<String, Entry<T>> services = new ConcurrentHashMap<>();
+    private final Collection<ServiceCache<T>> caches = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    private final Collection<ServiceProvider<T>> providers = Collections.newSetFromMap(new ConcurrentHashMap<>());
     private final boolean watchInstances;
     private final ConnectionStateListener connectionStateListener = new ConnectionStateListener() {
         @Override
@@ -105,12 +104,28 @@ public class ServiceDiscoveryImpl<T> implements ServiceDiscovery<T> {
             InstanceSerializer<T> serializer,
             ServiceInstance<T> thisInstance,
             boolean watchInstances) {
+        this(client, new DiscoveryPathConstructorImpl(basePath), serializer, thisInstance, watchInstances);
+    }
+
+    /**
+     * @param client the client
+     * @param pathConstructor constructor for instance paths
+     * @param serializer serializer for instances (e.g. {@link JsonInstanceSerializer})
+     * @param thisInstance instance that represents the service that is running. The instance will get auto-registered
+     * @param watchInstances if true, watches for changes to locally registered instances
+     */
+    public ServiceDiscoveryImpl(
+            CuratorFramework client,
+            DiscoveryPathConstructor pathConstructor,
+            InstanceSerializer<T> serializer,
+            ServiceInstance<T> thisInstance,
+            boolean watchInstances) {
         this.watchInstances = watchInstances;
         this.client = Preconditions.checkNotNull(client, "client cannot be null");
-        this.basePath = Preconditions.checkNotNull(basePath, "basePath cannot be null");
+        this.pathConstructor = Preconditions.checkNotNull(pathConstructor, "pathConstructor cannot be null");
         this.serializer = Preconditions.checkNotNull(serializer, "serializer cannot be null");
         if (thisInstance != null) {
-            Entry<T> entry = new Entry<T>(thisInstance);
+            Entry<T> entry = new Entry<>(thisInstance);
             entry.cache = makeNodeCache(thisInstance);
             services.put(thisInstance.getId(), entry);
         }
@@ -261,7 +276,7 @@ public class ServiceDiscoveryImpl<T> implements ServiceDiscovery<T> {
      */
     @Override
     public Collection<String> queryForNames() throws Exception {
-        List<String> names = client.getChildren().forPath(basePath);
+        List<String> names = client.getChildren().forPath(pathConstructor.getBasePath());
         return ImmutableList.copyOf(names);
     }
 
@@ -317,8 +332,8 @@ public class ServiceDiscoveryImpl<T> implements ServiceDiscovery<T> {
         return client;
     }
 
-    String pathForName(String name) {
-        return ZKPaths.makePath(basePath, name);
+    String pathForName(String serviceName) {
+        return pathConstructor.getPathForInstances(serviceName);
     }
 
     InstanceSerializer<T> getSerializer() {
@@ -374,8 +389,8 @@ public class ServiceDiscoveryImpl<T> implements ServiceDiscovery<T> {
     }
 
     @VisibleForTesting
-    String pathForInstance(String name, String id) {
-        return ZKPaths.makePath(pathForName(name), id);
+    String pathForInstance(String serviceName, String instanceId) {
+        return ZKPaths.makePath(pathForName(serviceName), instanceId);
     }
 
     @VisibleForTesting
diff --git a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImplTest.java b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImplTest.java
new file mode 100644
index 00000000..5cbb4ece
--- /dev/null
+++ b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/DiscoveryPathConstructorImplTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.curator.x.discovery.details;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import org.junit.jupiter.api.Test;
+
+public class DiscoveryPathConstructorImplTest {
+    @Test
+    public void testCanonicalDiscoveryPathConstructor() {
+        assertThatThrownBy(() -> new DiscoveryPathConstructorImpl(null)).isInstanceOf(IllegalArgumentException.class);
+        assertThatThrownBy(() -> new DiscoveryPathConstructorImpl("")).isInstanceOf(IllegalArgumentException.class);
+        assertThatThrownBy(() -> new DiscoveryPathConstructorImpl("foo/bar"))
+                .isInstanceOf(IllegalArgumentException.class);
+        assertThatThrownBy(() -> new DiscoveryPathConstructorImpl("foo/bar/"))
+                .isInstanceOf(IllegalArgumentException.class);
+        assertThatThrownBy(() -> new DiscoveryPathConstructorImpl("/foo/bar/"))
+                .isInstanceOf(IllegalArgumentException.class);
+    }
+
+    @Test
+    public void testGetBasePath() {
+        assertThat(new DiscoveryPathConstructorImpl("/foo/bar").getBasePath()).isEqualTo("/foo/bar");
+        assertThat(new DiscoveryPathConstructorImpl("/").getBasePath()).isEqualTo("/");
+    }
+
+    @Test
+    public void testGetPathForInstances() {
+        DiscoveryPathConstructorImpl constructor = new DiscoveryPathConstructorImpl("/foo/bar");
+        assertThat(constructor.getPathForInstances("baz")).isEqualTo("/foo/bar/baz");
+        assertThat(constructor.getPathForInstances("")).isEqualTo("/foo/bar");
+        assertThat(constructor.getPathForInstances(null)).isEqualTo("/foo/bar");
+    }
+}
diff --git a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
index 01fa1273..c6ade78b 100644
--- a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
+++ b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
@@ -36,6 +36,7 @@ import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
 import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.utils.CloseableUtils;
+import org.apache.curator.x.discovery.DiscoveryPathConstructor;
 import org.apache.curator.x.discovery.ServiceDiscovery;
 import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
 import org.apache.curator.x.discovery.ServiceInstance;
@@ -261,6 +262,49 @@ public class TestServiceDiscovery extends BaseClassForTests {
         }
     }
 
+    private static class CustomPathConstructor implements DiscoveryPathConstructor {
+        @Override
+        public String getBasePath() {
+            return "/test";
+        }
+
+        @Override
+        public String getPathForInstances(String serviceName) {
+            return String.format("/test/%s/instances", serviceName);
+        }
+    }
+
+    @Test
+    public void testCustomPathConstructor() throws Exception {
+        CuratorFramework client = null;
+        ServiceDiscovery<String> discovery = null;
+        try {
+            client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+            client.start();
+
+            ServiceInstance<String> instance = ServiceInstance.<String>builder()
+                    .payload("thing")
+                    .name("test")
+                    .port(10064)
+                    .build();
+            discovery = ServiceDiscoveryBuilder.builder(String.class)
+                    .pathConstructor(new CustomPathConstructor())
+                    .client(client)
+                    .thisInstance(instance)
+                    .build();
+            discovery.start();
+
+            assertEquals(discovery.queryForNames(), Collections.singletonList("test"));
+
+            List<ServiceInstance<String>> list = Lists.newArrayList();
+            list.add(instance);
+            assertEquals(discovery.queryForInstances("test"), list);
+        } finally {
+            CloseableUtils.closeQuietly(discovery);
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
     @Test
     public void testNoServerOnStart() throws Exception {
         Timing timing = new Timing();