leo-lao commented on code in PR #11144: URL: https://github.com/apache/dolphinscheduler/pull/11144#discussion_r930607744
########## 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); Review Comment: The same [ too many write requests] issue as above ########## 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); Review Comment: Here there exists possible too many write requests to KVStore, this would cause low efficiency. I thank you can accumulate enough expired masters to remove, and finally update this to KVStore in batch, rather than do it in each iteration. ########## dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java: ########## @@ -0,0 +1,188 @@ +/* + * 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.utils.JSONUtils; +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.Registry; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; +import org.apache.dolphinscheduler.spi.utils.StringUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import javax.annotation.PostConstruct; + +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.stereotype.Component; + +import com.alipay.sofa.jraft.option.NodeOptions; +import com.alipay.sofa.jraft.rhea.client.DefaultRheaKVStore; +import com.alipay.sofa.jraft.rhea.client.RheaKVStore; +import com.alipay.sofa.jraft.rhea.options.PlacementDriverOptions; +import com.alipay.sofa.jraft.rhea.options.RheaKVStoreOptions; +import com.alipay.sofa.jraft.rhea.options.StoreEngineOptions; +import com.alipay.sofa.jraft.rhea.options.configured.PlacementDriverOptionsConfigured; +import com.alipay.sofa.jraft.rhea.options.configured.RheaKVStoreOptionsConfigured; +import com.alipay.sofa.jraft.rhea.options.configured.RocksDBOptionsConfigured; +import com.alipay.sofa.jraft.rhea.options.configured.StoreEngineOptionsConfigured; +import com.alipay.sofa.jraft.rhea.storage.StorageType; +import com.alipay.sofa.jraft.rhea.util.concurrent.DistributedLock; +import com.alipay.sofa.jraft.util.Endpoint; + +import lombok.extern.slf4j.Slf4j; + +@Component +@Slf4j +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "raft") +public class RaftRegistry implements Registry { + + private final Map<String, DistributedLock<byte[]>> distributedLockMap = new ConcurrentHashMap<>(); + + private RheaKVStore kvStore; + + private RaftRegistryProperties properties; + + private EphemeralNodeManager ephemeralNodeManager; + + public RaftRegistry(RaftRegistryProperties properties) { + this.properties = properties; + //init RheaKVStore + final PlacementDriverOptions pdOpts = PlacementDriverOptionsConfigured.newConfigured() + .withFake(true) // use a fake pd + .config(); + NodeOptions nodeOptions = new NodeOptions(); + nodeOptions.setElectionTimeoutMs((int) properties.getElectionTimeout().toMillis()); + nodeOptions.setSnapshotIntervalSecs((int) properties.getSnapshotInterval().getSeconds()); + final StoreEngineOptions storeOpts = StoreEngineOptionsConfigured.newConfigured() + .withStorageType(StorageType.RocksDB) + .withRocksDBOptions(RocksDBOptionsConfigured.newConfigured().withDbPath(properties.getDbStorageDir()).config()) + .withRaftDataPath(properties.getLogStorageDir()) + .withServerAddress(new Endpoint(properties.getServerAddress(), properties.getServerPort())) + .withCommonNodeOptions(nodeOptions) + .config(); + final RheaKVStoreOptions opts = RheaKVStoreOptionsConfigured.newConfigured() + .withClusterName(properties.getClusterName()) + .withUseParallelCompress(true) + .withInitialServerList(properties.getServerAddressList()) + .withStoreEngineOptions(storeOpts) + .withPlacementDriverOptions(pdOpts) + .config(); + this.kvStore = new DefaultRheaKVStore(); + this.kvStore.init(opts); + log.info("kvStore started..."); + this.ephemeralNodeManager = new EphemeralNodeManager(properties, kvStore); + } + + @PostConstruct + public void start() { + ephemeralNodeManager.start(); + } + + @Override + public boolean subscribe(String path, SubscribeListener listener) { + return ephemeralNodeManager.addSubscribeListener(path, listener); + } + + @Override + public void unsubscribe(String path) { + ephemeralNodeManager.removeSubscribeListener(path); + } + + @Override + public void addConnectionStateListener(ConnectionListener listener) { + ephemeralNodeManager.addConnectionListener(listener); + } + + @Override + public String get(String key) { + return readUtf8(kvStore.bGet(key)); + } + + @Override + public void put(String key, String value, boolean deleteOnDisconnect) { + if (StringUtils.isBlank(value)) { + return; + } + readUtf8(kvStore.bGetAndPut(key, writeUtf8(value))); + ephemeralNodeManager.putHandler(key, value); + } + + @Override + public void delete(String key) { + kvStore.bDelete(key); + final DistributedLock<byte[]> distributedLock = distributedLockMap.get(key); + if (distributedLock != null) { + distributedLock.unlock(); + } + distributedLockMap.remove(key); + ephemeralNodeManager.deleteHandler(key); + + } + + @Override + public Collection<String> children(String key) { + final String result = readUtf8(kvStore.bGet(key)); + if (StringUtils.isEmpty(result)) { + return new ArrayList<>(); + } + final List<String> children = JSONUtils.toList(result, String.class); + children.sort(Comparator.reverseOrder()); + return children; + } + + @Override + public boolean exists(String key) { + return kvStore.bContainsKey(key); + } + + @Override + public boolean acquireLock(String key) { + final DistributedLock<byte[]> distributedLock = kvStore.getDistributedLock(key, properties.getDistributedLockTimeout().toMillis(), TimeUnit.MILLISECONDS); + final boolean lock = distributedLock.tryLock(); + if (lock) { + distributedLockMap.put(key, distributedLock); + } + return lock; Review Comment: Is it better to catch `Unable to get lock exception` and throw RegistryException manually? ########## 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); + } + } + } + Review Comment: Here I see `updateActiveMaster` and `updateActiveWorker` method, they will modify whole data under the key `/nodes/master` or `/nodes/worker`, rather than modify the specific record for expired masters found this time. And still, HeatBeat Task will also do like this. So there will be multiple threads in different hosts, trying to read and write the same key, which might cause collisions in distributed environment. My suggestion is : create one key for each master(thus 3 keys for 3 maters), rather than 1 key for 3 masters. -- 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]
