zhuxt2015 commented on code in PR #11144:
URL: https://github.com/apache/dolphinscheduler/pull/11144#discussion_r934004485


##########
dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.raft;
+
+import static com.alipay.sofa.jraft.util.BytesUtil.readUtf8;
+import static com.alipay.sofa.jraft.util.BytesUtil.writeUtf8;
+
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.utils.HeartBeat;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.NetUtils;
+import org.apache.dolphinscheduler.registry.api.ConnectionListener;
+import org.apache.dolphinscheduler.registry.api.ConnectionState;
+import org.apache.dolphinscheduler.registry.api.Event;
+import org.apache.dolphinscheduler.registry.api.SubscribeListener;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.alipay.sofa.jraft.rhea.client.RheaKVStore;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * 1. EphemeralNodeRefreshThread check current master node connection and 
check ephemeral node expire time
+ * 2. maintain Map<String, Long> activeMasterServers
+ * 3. maintain Map<String, Long> activeWorkerServers
+ * 4. maintain Map<Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS, List<String>> 
master servers
+ * 5. maintain Map<Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS, List<String>> 
worker servers
+ * 6. maintain Map<Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, 
List<String>> dead servers
+ */
+@Slf4j
+public class EphemeralNodeManager implements AutoCloseable {
+    private final List<ConnectionListener> connectionListeners = 
Collections.synchronizedList(new ArrayList<>());
+
+    private final Map<String, List<SubscribeListener>> dataSubScribeMap = new 
ConcurrentHashMap<>();
+
+    private RaftRegistryProperties properties;
+
+
+    private RheaKVStore kvStore;
+
+    public EphemeralNodeManager(RaftRegistryProperties properties, RheaKVStore 
kvStore) {
+        this.properties = properties;
+        this.kvStore = kvStore;
+    }
+
+    private final ScheduledExecutorService scheduledExecutorService = 
Executors.newScheduledThreadPool(
+            2,
+            new 
ThreadFactoryBuilder().setNameFormat("EphemeralNodeRefreshThread").setDaemon(true).build());
+
+    public void start() {
+        scheduledExecutorService.scheduleWithFixedDelay(new 
ConnectionCheckTask(),
+                properties.getListenerCheckInterval().toMillis(),
+                properties.getListenerCheckInterval().toMillis(),
+                TimeUnit.MILLISECONDS);
+        scheduledExecutorService.scheduleWithFixedDelay(new 
SubscribeCheckTask(),
+                properties.getListenerCheckInterval().toMillis(),
+                properties.getListenerCheckInterval().toMillis(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void close() {
+        connectionListeners.clear();
+        dataSubScribeMap.clear();
+        scheduledExecutorService.shutdown();
+    }
+
+    public void addConnectionListener(ConnectionListener listener) {
+        connectionListeners.add(listener);
+    }
+
+    public boolean addSubscribeListener(String path, SubscribeListener 
listener) {
+        return dataSubScribeMap.computeIfAbsent(path, k -> new 
ArrayList<>()).add(listener);
+    }
+
+    public void removeSubscribeListener(String path) {
+        dataSubScribeMap.remove(path);
+    }
+
+    private class ConnectionCheckTask implements Runnable {
+        private ConnectionState connectionState = null;
+
+        @Override
+        public void run() {
+            checkConnection();
+            checkActiveNode();
+        }
+
+        private void checkConnection() {
+            final String host = NetUtils.getHost();
+            if (getActiveMasterServers().keySet().stream().anyMatch(address -> 
address.split(Constants.COLON)[0].equals(host))) {
+                if (connectionState == null && !connectionListeners.isEmpty()) 
{
+                    triggerListener(ConnectionState.CONNECTED);
+                } else if (connectionState == ConnectionState.DISCONNECTED) {
+                    triggerListener(ConnectionState.RECONNECTED);
+                } else {
+                    triggerListener(ConnectionState.CONNECTED);
+                }
+                connectionState = ConnectionState.CONNECTED;
+            }
+        }
+
+        private void checkActiveNode() {
+            long expireTime = properties.getConnectionExpireFactor() * 
properties.getListenerCheckInterval().toMillis();
+            Map<String, Long> activeMasterServers = getActiveMasterServers();
+            for (Map.Entry<String, Long> entry : 
activeMasterServers.entrySet()) {
+                if ((System.currentTimeMillis() - entry.getValue()) > 
expireTime) {
+                    final String nodeAddress = entry.getKey();
+                    activeMasterServers.remove(nodeAddress);
+                    updateActiveMaster(activeMasterServers);
+                    addDeadServer(Constants.MASTER_TYPE, nodeAddress);
+                    if 
(nodeAddress.split(Constants.COLON)[0].equals(NetUtils.getHost())) {
+                        connectionState = ConnectionState.DISCONNECTED;
+                        triggerListener(ConnectionState.DISCONNECTED);
+                        removeNodeData(nodeAddress);
+                    }
+                    log.warn("Master server {} connect to raft cluster 
timeout, last heartbeat {}, timeout config {} ms",
+                            nodeAddress, 
convertTimeToString(entry.getValue()), expireTime);
+                }
+            }
+            Map<String, Long> activeWorkerServers = getActiveWorkerServers();
+            for (Map.Entry<String, Long> entry : 
activeWorkerServers.entrySet()) {
+                if ((System.currentTimeMillis() - entry.getValue()) > 
expireTime) {
+                    final String nodeAddress = entry.getKey();
+                    activeWorkerServers.remove(nodeAddress);
+                    updateActiveWorker(nodeAddress, activeWorkerServers);
+                    removeWorkerGroup(nodeAddress);
+                    addDeadServer(Constants.WORKER_TYPE, nodeAddress);
+                    removeNodeData(nodeAddress);
+                    log.warn("Worker server {} connect to raft cluster 
timeout, last heartbeat {}, timeout config {} ms",
+                            nodeAddress, 
convertTimeToString(entry.getValue()), expireTime);
+                }
+            }
+        }
+
+        private void triggerListener(ConnectionState connectionState) {
+            connectionListeners.forEach(listener -> 
listener.onUpdate(connectionState));
+        }
+    }
+
+    private class SubscribeCheckTask implements Runnable {
+
+        private final Map<String, String> nodeDataMap = new 
ConcurrentHashMap<>();
+
+        @Override
+        public void run() {
+            subscribeCheck();
+        }
+
+        private void subscribeCheck() {
+            if (dataSubScribeMap.isEmpty()) {
+                return;
+            }
+            final Map<String, String> currentNodeDataMap = getNodeDataMap();
+            // find the different
+            Map<String, String> addedData = new HashMap<>();
+            Map<String, String> deletedData = new HashMap<>();
+            Map<String, String> updatedData = new HashMap<>();
+            for (Map.Entry<String, String> entry : 
currentNodeDataMap.entrySet()) {
+                final String oldData = nodeDataMap.get(entry.getKey());
+                if (oldData == null) {
+                    addedData.put(entry.getKey(), entry.getValue());
+                } else {
+                    HeartBeat newHeartBeat = 
HeartBeat.decodeHeartBeat(entry.getValue());

Review Comment:
   Like zookeeper, subscribe listener compare node hearbeat to notify listener.



-- 
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]

Reply via email to