|
|
|
@ -17,9 +17,11 @@
|
|
|
|
|
|
|
|
|
|
package org.apache.dolphinscheduler.server.registry; |
|
|
|
|
|
|
|
|
|
import org.apache.commons.collections.CollectionUtils; |
|
|
|
|
import org.apache.curator.framework.CuratorFramework; |
|
|
|
|
|
|
|
|
|
import org.apache.curator.framework.recipes.cache.TreeCacheEvent; |
|
|
|
|
import org.apache.dolphinscheduler.common.utils.StringUtils; |
|
|
|
|
import org.apache.dolphinscheduler.service.zk.AbstractListener; |
|
|
|
|
import org.slf4j.Logger; |
|
|
|
|
import org.slf4j.LoggerFactory; |
|
|
|
@ -30,9 +32,12 @@ import org.springframework.stereotype.Service;
|
|
|
|
|
import java.util.Collections; |
|
|
|
|
import java.util.HashSet; |
|
|
|
|
import java.util.Set; |
|
|
|
|
import java.util.concurrent.ConcurrentHashMap; |
|
|
|
|
import java.util.concurrent.locks.Lock; |
|
|
|
|
import java.util.concurrent.locks.ReentrantLock; |
|
|
|
|
|
|
|
|
|
import static org.apache.dolphinscheduler.common.Constants.DEFAULT_WORKER_GROUP; |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* zookeeper node manager |
|
|
|
|
*/ |
|
|
|
@ -47,14 +52,14 @@ public class ZookeeperNodeManager implements InitializingBean {
|
|
|
|
|
private final Lock masterLock = new ReentrantLock(); |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* worker lock |
|
|
|
|
* worker group lock |
|
|
|
|
*/ |
|
|
|
|
private final Lock workerLock = new ReentrantLock(); |
|
|
|
|
private final Lock workerGroupLock = new ReentrantLock(); |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* worker nodes |
|
|
|
|
* worker group nodes |
|
|
|
|
*/ |
|
|
|
|
private final Set<String> workerNodes = new HashSet<>(); |
|
|
|
|
private final ConcurrentHashMap<String, Set<String>> workerGroupNodes = new ConcurrentHashMap<>(); |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* master nodes |
|
|
|
@ -84,7 +89,7 @@ public class ZookeeperNodeManager implements InitializingBean {
|
|
|
|
|
/** |
|
|
|
|
* init WorkerNodeListener listener |
|
|
|
|
*/ |
|
|
|
|
registryCenter.getZookeeperCachedOperator().addListener(new WorkerNodeListener()); |
|
|
|
|
registryCenter.getZookeeperCachedOperator().addListener(new WorkerGroupNodeListener()); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
@ -98,39 +103,55 @@ public class ZookeeperNodeManager implements InitializingBean {
|
|
|
|
|
syncMasterNodes(masterNodes); |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* worker nodes from zookeeper |
|
|
|
|
* worker group nodes from zookeeper |
|
|
|
|
*/ |
|
|
|
|
Set<String> workersNodes = registryCenter.getWorkerNodesDirectly(); |
|
|
|
|
syncWorkerNodes(workersNodes); |
|
|
|
|
Set<String> workerGroups = registryCenter.getWorkerGroupDirectly(); |
|
|
|
|
for(String workerGroup : workerGroups){ |
|
|
|
|
syncWorkerGroupNodes(workerGroup, registryCenter.getWorkerGroupNodesDirectly(workerGroup)); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* worker node listener |
|
|
|
|
* worker group node listener |
|
|
|
|
*/ |
|
|
|
|
class WorkerNodeListener extends AbstractListener { |
|
|
|
|
class WorkerGroupNodeListener extends AbstractListener { |
|
|
|
|
|
|
|
|
|
@Override |
|
|
|
|
protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) { |
|
|
|
|
if(registryCenter.isWorkerPath(path)){ |
|
|
|
|
try { |
|
|
|
|
if (event.getType() == TreeCacheEvent.Type.NODE_ADDED) { |
|
|
|
|
logger.info("worker node : {} added.", path); |
|
|
|
|
logger.info("worker group node : {} added.", path); |
|
|
|
|
String group = parseGroup(path); |
|
|
|
|
Set<String> workerNodes = workerGroupNodes.getOrDefault(group, new HashSet<>()); |
|
|
|
|
Set<String> previousNodes = new HashSet<>(workerNodes); |
|
|
|
|
Set<String> currentNodes = registryCenter.getWorkerNodesDirectly(); |
|
|
|
|
syncWorkerNodes(currentNodes); |
|
|
|
|
Set<String> currentNodes = registryCenter.getWorkerGroupNodesDirectly(group); |
|
|
|
|
logger.info("currentNodes : {}", currentNodes); |
|
|
|
|
syncWorkerGroupNodes(group, currentNodes); |
|
|
|
|
} else if (event.getType() == TreeCacheEvent.Type.NODE_REMOVED) { |
|
|
|
|
logger.info("worker node : {} down.", path); |
|
|
|
|
logger.info("worker group node : {} down.", path); |
|
|
|
|
String group = parseGroup(path); |
|
|
|
|
Set<String> workerNodes = workerGroupNodes.getOrDefault(group, new HashSet<>()); |
|
|
|
|
Set<String> previousNodes = new HashSet<>(workerNodes); |
|
|
|
|
Set<String> currentNodes = registryCenter.getWorkerNodesDirectly(); |
|
|
|
|
syncWorkerNodes(currentNodes); |
|
|
|
|
Set<String> currentNodes = registryCenter.getWorkerGroupNodesDirectly(group); |
|
|
|
|
syncWorkerGroupNodes(group, currentNodes); |
|
|
|
|
} |
|
|
|
|
} catch (IllegalArgumentException ignore) { |
|
|
|
|
logger.warn(ignore.getMessage()); |
|
|
|
|
} catch (Exception ex) { |
|
|
|
|
logger.error("WorkerListener capture data change and get data failed", ex); |
|
|
|
|
logger.error("WorkerGroupListener capture data change and get data failed", ex); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private String parseGroup(String path){ |
|
|
|
|
String[] parts = path.split("\\/"); |
|
|
|
|
if(parts.length != 6){ |
|
|
|
|
throw new IllegalArgumentException(String.format("worker group path : %s is not valid, ignore", path)); |
|
|
|
|
} |
|
|
|
|
String group = parts[4]; |
|
|
|
|
return group; |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
@ -189,29 +210,42 @@ public class ZookeeperNodeManager implements InitializingBean {
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* sync worker nodes |
|
|
|
|
* @param nodes worker nodes |
|
|
|
|
* sync worker group nodes |
|
|
|
|
* @param workerGroup |
|
|
|
|
* @param nodes |
|
|
|
|
*/ |
|
|
|
|
private void syncWorkerNodes(Set<String> nodes){ |
|
|
|
|
workerLock.lock(); |
|
|
|
|
private void syncWorkerGroupNodes(String workerGroup, Set<String> nodes){ |
|
|
|
|
workerGroupLock.lock(); |
|
|
|
|
try { |
|
|
|
|
workerGroup = workerGroup.toLowerCase(); |
|
|
|
|
Set<String> workerNodes = workerGroupNodes.getOrDefault(workerGroup, new HashSet<>()); |
|
|
|
|
workerNodes.clear(); |
|
|
|
|
workerNodes.addAll(nodes); |
|
|
|
|
workerGroupNodes.put(workerGroup, workerNodes); |
|
|
|
|
} finally { |
|
|
|
|
workerLock.unlock(); |
|
|
|
|
workerGroupLock.unlock(); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
* get worker nodes |
|
|
|
|
* @return worker nodes |
|
|
|
|
* get worker group nodes |
|
|
|
|
* @param workerGroup |
|
|
|
|
* @return |
|
|
|
|
*/ |
|
|
|
|
public Set<String> getWorkerNodes(){ |
|
|
|
|
workerLock.lock(); |
|
|
|
|
public Set<String> getWorkerGroupNodes(String workerGroup){ |
|
|
|
|
workerGroupLock.lock(); |
|
|
|
|
try { |
|
|
|
|
return Collections.unmodifiableSet(workerNodes); |
|
|
|
|
if(StringUtils.isEmpty(workerGroup)){ |
|
|
|
|
workerGroup = DEFAULT_WORKER_GROUP; |
|
|
|
|
} |
|
|
|
|
workerGroup = workerGroup.toLowerCase(); |
|
|
|
|
Set<String> nodes = workerGroupNodes.get(workerGroup); |
|
|
|
|
if(CollectionUtils.isNotEmpty(nodes)){ |
|
|
|
|
return Collections.unmodifiableSet(nodes); |
|
|
|
|
} |
|
|
|
|
return nodes; |
|
|
|
|
} finally { |
|
|
|
|
workerLock.unlock(); |
|
|
|
|
workerGroupLock.unlock(); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|