caishunfeng commented on code in PR #10981: URL: https://github.com/apache/dolphinscheduler/pull/10981#discussion_r933918952
########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.dolphinscheduler.plugin.registry.etcd; + +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.Registry; +import org.apache.dolphinscheduler.registry.api.RegistryException; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import javax.annotation.PostConstruct; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.stereotype.Component; + +import com.google.common.base.Splitter; +import com.google.common.base.Strings; + +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.Client; +import io.etcd.jetcd.ClientBuilder; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Lease; +import io.etcd.jetcd.Lock; +import io.etcd.jetcd.Util; +import io.etcd.jetcd.Watch; +import io.etcd.jetcd.options.DeleteOption; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.PutOption; +import io.etcd.jetcd.options.WatchOption; +import io.etcd.jetcd.support.Observers; +import io.etcd.jetcd.watch.WatchEvent; + +/** + * This is one of the implementation of {@link Registry}, with this implementation, you need to rely on Etcd CLuster to + * store the DolphinScheduler master/worker's metadata and do the server registry/unRegistry. + */ +@Component +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "etcd") +public class EtcdRegistry implements Registry { + private static Logger LOGGER = LoggerFactory.getLogger(EtcdRegistry.class); + private final Client client; + private EtcdConnectionStateListener etcdConnectionStateListener; + public static final String FOLDER_SEPARATOR = "/"; + // save the lock info for thread + // key:lockKey Value:leaseId + private static final ThreadLocal<Map<String, Long>> threadLocalLockMap = new ThreadLocal<>(); + + private final Map<String, Watch.Watcher> watcherMap = new ConcurrentHashMap<>(); + + private static Long TIME_TO_LIVE_SECONDS = 30L; + public EtcdRegistry(EtcdRegistryProperties registryProperties) { + LOGGER.info("Starting Etcd Registry..."); + ClientBuilder clientBuilder = Client.builder() + .endpoints(Util.toURIs(Splitter.on(",").trimResults().splitToList(registryProperties.getEndpoints()))) + .namespace(byteSequence(registryProperties.getNamespace())) + .connectTimeout(registryProperties.getConnectionTimeout()) + .retryChronoUnit(ChronoUnit.MILLIS) + .retryDelay(registryProperties.getRetryDelay().toMillis()) + .retryMaxDelay(registryProperties.getRetryMaxDelay().toMillis()) + .retryMaxDuration(registryProperties.getRetryMaxDuration()); + if (!Strings.isNullOrEmpty(registryProperties.getUser()) && (!Strings.isNullOrEmpty(registryProperties.getPassword()))) { Review Comment: ```suggestion if (StringUtils.isNotEmpty(registryProperties.getUser()) && (StringUtils.isNotEmpty(registryProperties.getPassword()))) { ``` ########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/test/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistryTest.java: ########## @@ -0,0 +1,141 @@ +/* + * 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.dolphinscheduler.plugin.registry.etcd; + +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.etcd.jetcd.test.EtcdClusterExtension; + +public class EtcdRegistryTest { Review Comment: :+1: ########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.dolphinscheduler.plugin.registry.etcd; + +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.Registry; +import org.apache.dolphinscheduler.registry.api.RegistryException; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import javax.annotation.PostConstruct; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.stereotype.Component; + +import com.google.common.base.Splitter; +import com.google.common.base.Strings; + +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.Client; +import io.etcd.jetcd.ClientBuilder; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Lease; +import io.etcd.jetcd.Lock; +import io.etcd.jetcd.Util; +import io.etcd.jetcd.Watch; +import io.etcd.jetcd.options.DeleteOption; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.PutOption; +import io.etcd.jetcd.options.WatchOption; +import io.etcd.jetcd.support.Observers; +import io.etcd.jetcd.watch.WatchEvent; + +/** + * This is one of the implementation of {@link Registry}, with this implementation, you need to rely on Etcd CLuster to + * store the DolphinScheduler master/worker's metadata and do the server registry/unRegistry. + */ +@Component +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "etcd") +public class EtcdRegistry implements Registry { + private static Logger LOGGER = LoggerFactory.getLogger(EtcdRegistry.class); + private final Client client; + private EtcdConnectionStateListener etcdConnectionStateListener; + public static final String FOLDER_SEPARATOR = "/"; + // save the lock info for thread + // key:lockKey Value:leaseId + private static final ThreadLocal<Map<String, Long>> threadLocalLockMap = new ThreadLocal<>(); + + private final Map<String, Watch.Watcher> watcherMap = new ConcurrentHashMap<>(); + + private static Long TIME_TO_LIVE_SECONDS = 30L; + public EtcdRegistry(EtcdRegistryProperties registryProperties) { + LOGGER.info("Starting Etcd Registry..."); + ClientBuilder clientBuilder = Client.builder() + .endpoints(Util.toURIs(Splitter.on(",").trimResults().splitToList(registryProperties.getEndpoints()))) + .namespace(byteSequence(registryProperties.getNamespace())) + .connectTimeout(registryProperties.getConnectionTimeout()) + .retryChronoUnit(ChronoUnit.MILLIS) + .retryDelay(registryProperties.getRetryDelay().toMillis()) + .retryMaxDelay(registryProperties.getRetryMaxDelay().toMillis()) + .retryMaxDuration(registryProperties.getRetryMaxDuration()); + if (!Strings.isNullOrEmpty(registryProperties.getUser()) && (!Strings.isNullOrEmpty(registryProperties.getPassword()))) { + clientBuilder.user(byteSequence(registryProperties.getUser())); + clientBuilder.password(byteSequence(registryProperties.getPassword())); + } + if (!Strings.isNullOrEmpty(registryProperties.getLoadBalancerPolicy())) { + clientBuilder.loadBalancerPolicy(registryProperties.getLoadBalancerPolicy()); + } + if (!Strings.isNullOrEmpty(registryProperties.getAuthority())) { + clientBuilder.authority(registryProperties.getAuthority()); + } + client = clientBuilder.build(); + LOGGER.info("Started Etcd Registry..."); + etcdConnectionStateListener = new EtcdConnectionStateListener(client); + } + + /** + * Start the etcd Connection stateListeer + */ + @PostConstruct + public void start() { + LOGGER.info("Starting Etcd ConnectionListener..."); + etcdConnectionStateListener.start(); + LOGGER.info("Started Etcd ConnectionListener..."); + } + + /** + * + * @param path The prefix of the key being listened to + * @param listener + * @return if subcribe Returns true if no exception was thrown + */ + @Override + public boolean subscribe(String path, SubscribeListener listener) { + try { + ByteSequence watchKey = byteSequence(path); + WatchOption watchOption = WatchOption.newBuilder().isPrefix(true).build(); + watcherMap.computeIfAbsent(path, $ -> client.getWatchClient().watch(watchKey, watchOption,watchResponse -> { + for (WatchEvent event : watchResponse.getEvents()) { + listener.notify(new EventAdaptor(event, path)); + } + })); + } catch (Exception e) { + throw new RegistryException("Failed to subscribe listener for key: " + path, e); + } + return true; + } + + /** + * @throws throws an exception if the unsubscribe path does not exist + * @param path The prefix of the key being listened to + */ + @Override + public void unsubscribe(String path) { + try { + watcherMap.get(path).close(); + watcherMap.remove(path); + } catch (Exception e) { + throw new RegistryException("Failed to unsubscribe listener for key: " + path, e); + } + } + + @Override + public void addConnectionStateListener(ConnectionListener listener) { + etcdConnectionStateListener.addConnectionListener(listener); + } + + /** + * + * @param key + * @return Returns the value corresponding to the key + * @throws throws an exception if the key does not exist + */ + @Override + public String get(String key) { + try { + List<KeyValue> keyValues = client.getKVClient().get(byteSequence(key)).get().getKvs(); + return keyValues.iterator().next().getValue().toString(StandardCharsets.UTF_8); + } catch (InterruptedException | ExecutionException e) { + Thread.currentThread().interrupt(); + throw new RegistryException("etcd get data error", e); + } + } + + /** + * + * @param key + * @param value + * @param deleteOnDisconnect Does the put data disappear when the client disconnects + */ + @Override + public void put(String key, String value, boolean deleteOnDisconnect) { + try { + if (deleteOnDisconnect) { + // keep the key by lease, if disconnected, the lease will expire and the key will delete + long leaseId = client.getLeaseClient().grant(TIME_TO_LIVE_SECONDS).get().getID(); + client.getLeaseClient().keepAlive(leaseId, Observers.observer(response -> { + })); + PutOption putOption = PutOption.newBuilder().withLeaseId(leaseId).build(); + client.getKVClient().put(byteSequence(key), byteSequence(value),putOption).get(); + } else { + client.getKVClient().put(byteSequence(key), byteSequence(value)).get(); + } + } catch (InterruptedException | ExecutionException e) { + Thread.currentThread().interrupt(); + throw new RegistryException("Failed to put registry key: " + key, e); + } + } + + /** + * delete all keys that contain the prefix + * @param key the prrefix Review Comment: ```suggestion * @param the key prefix ``` ########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.dolphinscheduler.plugin.registry.etcd; + +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.Registry; +import org.apache.dolphinscheduler.registry.api.RegistryException; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import javax.annotation.PostConstruct; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.stereotype.Component; + +import com.google.common.base.Splitter; +import com.google.common.base.Strings; + +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.Client; +import io.etcd.jetcd.ClientBuilder; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Lease; +import io.etcd.jetcd.Lock; +import io.etcd.jetcd.Util; +import io.etcd.jetcd.Watch; +import io.etcd.jetcd.options.DeleteOption; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.PutOption; +import io.etcd.jetcd.options.WatchOption; +import io.etcd.jetcd.support.Observers; +import io.etcd.jetcd.watch.WatchEvent; + +/** + * This is one of the implementation of {@link Registry}, with this implementation, you need to rely on Etcd CLuster to + * store the DolphinScheduler master/worker's metadata and do the server registry/unRegistry. + */ +@Component +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "etcd") +public class EtcdRegistry implements Registry { + private static Logger LOGGER = LoggerFactory.getLogger(EtcdRegistry.class); + private final Client client; + private EtcdConnectionStateListener etcdConnectionStateListener; + public static final String FOLDER_SEPARATOR = "/"; + // save the lock info for thread + // key:lockKey Value:leaseId + private static final ThreadLocal<Map<String, Long>> threadLocalLockMap = new ThreadLocal<>(); + + private final Map<String, Watch.Watcher> watcherMap = new ConcurrentHashMap<>(); + + private static Long TIME_TO_LIVE_SECONDS = 30L; + public EtcdRegistry(EtcdRegistryProperties registryProperties) { + LOGGER.info("Starting Etcd Registry..."); + ClientBuilder clientBuilder = Client.builder() + .endpoints(Util.toURIs(Splitter.on(",").trimResults().splitToList(registryProperties.getEndpoints()))) + .namespace(byteSequence(registryProperties.getNamespace())) + .connectTimeout(registryProperties.getConnectionTimeout()) + .retryChronoUnit(ChronoUnit.MILLIS) + .retryDelay(registryProperties.getRetryDelay().toMillis()) + .retryMaxDelay(registryProperties.getRetryMaxDelay().toMillis()) + .retryMaxDuration(registryProperties.getRetryMaxDuration()); + if (!Strings.isNullOrEmpty(registryProperties.getUser()) && (!Strings.isNullOrEmpty(registryProperties.getPassword()))) { + clientBuilder.user(byteSequence(registryProperties.getUser())); + clientBuilder.password(byteSequence(registryProperties.getPassword())); + } + if (!Strings.isNullOrEmpty(registryProperties.getLoadBalancerPolicy())) { + clientBuilder.loadBalancerPolicy(registryProperties.getLoadBalancerPolicy()); + } + if (!Strings.isNullOrEmpty(registryProperties.getAuthority())) { + clientBuilder.authority(registryProperties.getAuthority()); + } + client = clientBuilder.build(); + LOGGER.info("Started Etcd Registry..."); + etcdConnectionStateListener = new EtcdConnectionStateListener(client); + } + + /** + * Start the etcd Connection stateListeer + */ + @PostConstruct + public void start() { + LOGGER.info("Starting Etcd ConnectionListener..."); + etcdConnectionStateListener.start(); + LOGGER.info("Started Etcd ConnectionListener..."); + } + + /** + * + * @param path The prefix of the key being listened to + * @param listener + * @return if subcribe Returns true if no exception was thrown + */ + @Override + public boolean subscribe(String path, SubscribeListener listener) { + try { + ByteSequence watchKey = byteSequence(path); + WatchOption watchOption = WatchOption.newBuilder().isPrefix(true).build(); + watcherMap.computeIfAbsent(path, $ -> client.getWatchClient().watch(watchKey, watchOption,watchResponse -> { + for (WatchEvent event : watchResponse.getEvents()) { + listener.notify(new EventAdaptor(event, path)); + } + })); + } catch (Exception e) { + throw new RegistryException("Failed to subscribe listener for key: " + path, e); + } + return true; + } + + /** + * @throws throws an exception if the unsubscribe path does not exist + * @param path The prefix of the key being listened to + */ + @Override + public void unsubscribe(String path) { + try { + watcherMap.get(path).close(); + watcherMap.remove(path); + } catch (Exception e) { + throw new RegistryException("Failed to unsubscribe listener for key: " + path, e); + } + } + + @Override + public void addConnectionStateListener(ConnectionListener listener) { + etcdConnectionStateListener.addConnectionListener(listener); + } + + /** + * + * @param key + * @return Returns the value corresponding to the key + * @throws throws an exception if the key does not exist + */ + @Override + public String get(String key) { + try { + List<KeyValue> keyValues = client.getKVClient().get(byteSequence(key)).get().getKvs(); + return keyValues.iterator().next().getValue().toString(StandardCharsets.UTF_8); + } catch (InterruptedException | ExecutionException e) { + Thread.currentThread().interrupt(); + throw new RegistryException("etcd get data error", e); + } + } + + /** + * + * @param key + * @param value + * @param deleteOnDisconnect Does the put data disappear when the client disconnects + */ + @Override + public void put(String key, String value, boolean deleteOnDisconnect) { + try { + if (deleteOnDisconnect) { + // keep the key by lease, if disconnected, the lease will expire and the key will delete + long leaseId = client.getLeaseClient().grant(TIME_TO_LIVE_SECONDS).get().getID(); + client.getLeaseClient().keepAlive(leaseId, Observers.observer(response -> { + })); + PutOption putOption = PutOption.newBuilder().withLeaseId(leaseId).build(); + client.getKVClient().put(byteSequence(key), byteSequence(value),putOption).get(); + } else { + client.getKVClient().put(byteSequence(key), byteSequence(value)).get(); + } + } catch (InterruptedException | ExecutionException e) { + Thread.currentThread().interrupt(); + throw new RegistryException("Failed to put registry key: " + key, e); + } + } + + /** + * delete all keys that contain the prefix + * @param key the prrefix + */ + @Override + public void delete(String key) { Review Comment: It's better to use the more precise method name, such as `deleteByPrefix(String keyPrefix)` ########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.dolphinscheduler.plugin.registry.etcd; + +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.Registry; +import org.apache.dolphinscheduler.registry.api.RegistryException; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import javax.annotation.PostConstruct; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.stereotype.Component; + +import com.google.common.base.Splitter; +import com.google.common.base.Strings; + +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.Client; +import io.etcd.jetcd.ClientBuilder; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Lease; +import io.etcd.jetcd.Lock; +import io.etcd.jetcd.Util; +import io.etcd.jetcd.Watch; +import io.etcd.jetcd.options.DeleteOption; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.PutOption; +import io.etcd.jetcd.options.WatchOption; +import io.etcd.jetcd.support.Observers; +import io.etcd.jetcd.watch.WatchEvent; + +/** + * This is one of the implementation of {@link Registry}, with this implementation, you need to rely on Etcd CLuster to + * store the DolphinScheduler master/worker's metadata and do the server registry/unRegistry. + */ +@Component +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "etcd") +public class EtcdRegistry implements Registry { + private static Logger LOGGER = LoggerFactory.getLogger(EtcdRegistry.class); + private final Client client; + private EtcdConnectionStateListener etcdConnectionStateListener; + public static final String FOLDER_SEPARATOR = "/"; + // save the lock info for thread + // key:lockKey Value:leaseId + private static final ThreadLocal<Map<String, Long>> threadLocalLockMap = new ThreadLocal<>(); + + private final Map<String, Watch.Watcher> watcherMap = new ConcurrentHashMap<>(); + + private static Long TIME_TO_LIVE_SECONDS = 30L; + public EtcdRegistry(EtcdRegistryProperties registryProperties) { + LOGGER.info("Starting Etcd Registry..."); + ClientBuilder clientBuilder = Client.builder() + .endpoints(Util.toURIs(Splitter.on(",").trimResults().splitToList(registryProperties.getEndpoints()))) + .namespace(byteSequence(registryProperties.getNamespace())) + .connectTimeout(registryProperties.getConnectionTimeout()) + .retryChronoUnit(ChronoUnit.MILLIS) + .retryDelay(registryProperties.getRetryDelay().toMillis()) + .retryMaxDelay(registryProperties.getRetryMaxDelay().toMillis()) + .retryMaxDuration(registryProperties.getRetryMaxDuration()); + if (!Strings.isNullOrEmpty(registryProperties.getUser()) && (!Strings.isNullOrEmpty(registryProperties.getPassword()))) { + clientBuilder.user(byteSequence(registryProperties.getUser())); + clientBuilder.password(byteSequence(registryProperties.getPassword())); + } + if (!Strings.isNullOrEmpty(registryProperties.getLoadBalancerPolicy())) { + clientBuilder.loadBalancerPolicy(registryProperties.getLoadBalancerPolicy()); + } + if (!Strings.isNullOrEmpty(registryProperties.getAuthority())) { + clientBuilder.authority(registryProperties.getAuthority()); + } + client = clientBuilder.build(); + LOGGER.info("Started Etcd Registry..."); + etcdConnectionStateListener = new EtcdConnectionStateListener(client); + } + + /** + * Start the etcd Connection stateListeer + */ + @PostConstruct + public void start() { + LOGGER.info("Starting Etcd ConnectionListener..."); + etcdConnectionStateListener.start(); + LOGGER.info("Started Etcd ConnectionListener..."); + } + + /** + * + * @param path The prefix of the key being listened to + * @param listener + * @return if subcribe Returns true if no exception was thrown + */ + @Override + public boolean subscribe(String path, SubscribeListener listener) { + try { + ByteSequence watchKey = byteSequence(path); + WatchOption watchOption = WatchOption.newBuilder().isPrefix(true).build(); + watcherMap.computeIfAbsent(path, $ -> client.getWatchClient().watch(watchKey, watchOption,watchResponse -> { + for (WatchEvent event : watchResponse.getEvents()) { + listener.notify(new EventAdaptor(event, path)); + } + })); + } catch (Exception e) { + throw new RegistryException("Failed to subscribe listener for key: " + path, e); + } + return true; + } + + /** + * @throws throws an exception if the unsubscribe path does not exist + * @param path The prefix of the key being listened to + */ + @Override + public void unsubscribe(String path) { + try { + watcherMap.get(path).close(); + watcherMap.remove(path); + } catch (Exception e) { + throw new RegistryException("Failed to unsubscribe listener for key: " + path, e); + } + } + + @Override + public void addConnectionStateListener(ConnectionListener listener) { + etcdConnectionStateListener.addConnectionListener(listener); + } + + /** + * + * @param key + * @return Returns the value corresponding to the key + * @throws throws an exception if the key does not exist + */ + @Override + public String get(String key) { + try { + List<KeyValue> keyValues = client.getKVClient().get(byteSequence(key)).get().getKvs(); + return keyValues.iterator().next().getValue().toString(StandardCharsets.UTF_8); + } catch (InterruptedException | ExecutionException e) { Review Comment: If throw ExecutionException, should we call `Thread.currentThread().interrupt()`? It's better to handle separately, WDYT? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
