From 0afd8573f1d780b5f31c968f10c2580cbaa4004c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Sun, 24 Jul 2022 21:19:37 +0800 Subject: [PATCH 1/9] remove zookeeper --- dolphinscheduler-bom/pom.xml | 12 + dolphinscheduler-master/pom.xml | 8 + .../dolphinscheduler-registry-all/pom.xml | 5 + .../dolphinscheduler-registry-raft/README.md | 32 ++ .../dolphinscheduler-registry-raft/pom.xml | 33 ++ .../registry/raft/EphemeralNodeManager.java | 405 ++++++++++++++++++ .../plugin/registry/raft/RaftRegistry.java | 188 ++++++++ .../registry/raft/RaftRegistryProperties.java | 47 ++ .../dolphinscheduler-registry-plugins/pom.xml | 1 + 9 files changed, 731 insertions(+) create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java diff --git a/dolphinscheduler-bom/pom.xml b/dolphinscheduler-bom/pom.xml index 0599d5510eb9..b14a1aa1b4b2 100644 --- a/dolphinscheduler-bom/pom.xml +++ b/dolphinscheduler-bom/pom.xml @@ -85,6 +85,7 @@ 3.14.9 2.7.0 2021.0.3 + 1.3.11 @@ -593,6 +594,17 @@ json-path ${json-path.version} + + + com.alipay.sofa + jraft-core + ${jraft.version} + + + com.alipay.sofa + jraft-rheakv-core + ${jraft.version} + org.springframework.cloud diff --git a/dolphinscheduler-master/pom.xml b/dolphinscheduler-master/pom.xml index 558741fea196..0c8a1b9b1e27 100644 --- a/dolphinscheduler-master/pom.xml +++ b/dolphinscheduler-master/pom.xml @@ -133,6 +133,10 @@ log4j log4j + + com.google.protobuf + protobuf-java + @@ -225,6 +229,10 @@ jersey-core com.sun.jersey + + com.google.protobuf + protobuf-java + diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-all/pom.xml b/dolphinscheduler-registry/dolphinscheduler-registry-all/pom.xml index de853f0eed76..8b439a9b2b3e 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-all/pom.xml +++ b/dolphinscheduler-registry/dolphinscheduler-registry-all/pom.xml @@ -38,5 +38,10 @@ dolphinscheduler-registry-mysql ${project.version} + + org.apache.dolphinscheduler + dolphinscheduler-registry-raft + ${project.version} + \ No newline at end of file diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md new file mode 100644 index 000000000000..962d544b0f55 --- /dev/null +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md @@ -0,0 +1,32 @@ +# Introduction + +This module is the RAFT consensus algorithm registry plugin module, this plugin will use raft cluster as the registry center. + +# How to use + +If you want to set the registry center as raft, + +you need to set the registry properties in master/worker/api's appplication.yml, + +please remember change the server-port in appplication.yml. + +In PRO environment, worker and api add `/learner` suffix in `server-address-list` + + +```yaml +registry: + type: raft + cluster-name: dolphinscheduler + server-address-list: 127.0.0.1:8181,127.0.0.1:8182/learner,127.0.0.1:8183/learner + log-storage-dir: raft-data/ + db-storage-dir: raft-db/ + election-timeout: 1000ms + snapshot-interval: 1800s + listener-check-interval: 2s + connection-expire-factor: 2 + server-address: 127.0.0.1 + server-port: 8181 +``` + +Then you can start your DolphinScheduler cluster, your cluster will use raft cluster as registry center to +store server metadata. \ No newline at end of file diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml new file mode 100644 index 000000000000..1311870dfd2e --- /dev/null +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml @@ -0,0 +1,33 @@ + + + + + org.apache.dolphinscheduler + dolphinscheduler-registry-plugins + dev-SNAPSHOT + + 4.0.0 + + dolphinscheduler-registry-raft + + + + org.apache.dolphinscheduler + dolphinscheduler-registry-api + + + org.apache.dolphinscheduler + dolphinscheduler-common + + + com.alipay.sofa + jraft-core + + + com.alipay.sofa + jraft-rheakv-core + + + + diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java new file mode 100644 index 000000000000..abb9aec1c3ce --- /dev/null +++ b/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 activeMasterServers + * 3. maintain Map activeWorkerServers + * 4. maintain Map> master servers + * 5. maintain Map> worker servers + * 6. maintain Map> dead servers + */ +@Slf4j +public class EphemeralNodeManager implements AutoCloseable { + private final List connectionListeners = Collections.synchronizedList(new ArrayList<>()); + + private final Map> 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 activeMasterServers = getActiveMasterServers(); + for (Map.Entry 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 activeWorkerServers = getActiveWorkerServers(); + for (Map.Entry 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 nodeDataMap = new ConcurrentHashMap<>(); + + @Override + public void run() { + subscribeCheck(); + } + + private void subscribeCheck() { + if (dataSubScribeMap.isEmpty()) { + return; + } + final Map currentNodeDataMap = getNodeDataMap(); + // find the different + Map addedData = new HashMap<>(); + Map deletedData = new HashMap<>(); + Map updatedData = new HashMap<>(); + for (Map.Entry 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()); + HeartBeat oldHeartBeat = HeartBeat.decodeHeartBeat(oldData); + if (newHeartBeat != null && newHeartBeat.getReportTime() != oldHeartBeat.getReportTime()) { + updatedData.put(entry.getKey(), entry.getValue()); + } + } + } + for (Map.Entry entry : nodeDataMap.entrySet()) { + if (!currentNodeDataMap.containsKey(entry.getKey())) { + deletedData.put(entry.getKey(), entry.getValue()); + } + } + nodeDataMap.clear(); + nodeDataMap.putAll(currentNodeDataMap); + // trigger listener + for (Map.Entry> entry : dataSubScribeMap.entrySet()) { + String subscribeKey = entry.getKey(); + List subscribeListeners = entry.getValue(); + triggerListener(addedData, subscribeKey, subscribeListeners, Event.Type.ADD); + triggerListener(deletedData, subscribeKey, subscribeListeners, Event.Type.REMOVE); + triggerListener(updatedData, subscribeKey, subscribeListeners, Event.Type.UPDATE); + } + + } + + private void triggerListener(Map nodeDataMap, String subscribeKey, List subscribeListeners, Event.Type type) { + for (Map.Entry entry : nodeDataMap.entrySet()) { + final String key = entry.getKey(); + if (key.startsWith(subscribeKey)) { + subscribeListeners.forEach(listener -> listener.notify(new Event(key, key, entry.getValue(), type))); + } + } + } + } + + public static String convertTimeToString(Long time) { + DateTimeFormatter ftf = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"); + return ftf.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(time), ZoneId.systemDefault())); + } + + public void putHandler(String key, String value) { + final String nodeAddress = key.substring(key.lastIndexOf(Constants.SINGLE_SLASH) + 1); + //update heart beat time and node set + if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS)) { + Map activeMasterServers = getActiveMasterServers(); + activeMasterServers.put(nodeAddress, System.currentTimeMillis()); + updateActiveMaster(activeMasterServers); + removeDeadServer(Constants.MASTER_TYPE, nodeAddress); + addNodeData(key, value); + } else if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)) { + Map activeWorkerServers = getActiveWorkerServers(); + activeWorkerServers.put(nodeAddress, System.currentTimeMillis()); + updateActiveWorker(key, activeWorkerServers); + addWorkerGroup(key); + removeDeadServer(Constants.WORKER_TYPE, nodeAddress); + addNodeData(key, value); + } else if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS)) { + final List deadServers = getDeadServers(); + if (!deadServers.contains(nodeAddress)) { + deadServers.add(nodeAddress); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); + } + } + + } + + public void deleteHandler(String key) { + final String nodeAddress = key.substring(key.lastIndexOf(Constants.SINGLE_SLASH) + 1); + if (key.contains(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS)) { + Map activeMasterServers = getActiveMasterServers(); + activeMasterServers.remove(nodeAddress); + updateActiveMaster(activeMasterServers); + removeNodeData(nodeAddress); + log.info("Raft registry remove master server {}", nodeAddress); + } else if (key.contains(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)) { + Map activeWorkerServers = getActiveWorkerServers(); + activeWorkerServers.remove(nodeAddress); + updateActiveWorker(key, activeWorkerServers); + removeWorkerGroup(nodeAddress); + removeNodeData(nodeAddress); + log.info("Raft registry remove worker server {}", nodeAddress); + } + } + + private void updateActiveMaster(Map activeNodes) { + kvStore.bPut(Constants.MASTER_TYPE, writeUtf8(JSONUtils.toJsonString(activeNodes))); + //Update the mapping of the master group and master node list + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS, writeUtf8(JSONUtils.toJsonString(activeNodes.keySet()))); + } + + private void updateActiveWorker(String key, Map activeNodes) { + kvStore.bPut(Constants.WORKER_TYPE, writeUtf8(JSONUtils.toJsonString(activeNodes))); + //Update the mapping of the worker group and worker node list + kvStore.bPut(key.substring(0, key.lastIndexOf(Constants.SINGLE_SLASH)), writeUtf8(JSONUtils.toJsonString(activeNodes.keySet()))); + } + + private void addDeadServer(String nodeType, String nodeAddress) { + final String deadServerAddress = getDeadServerSuffix(nodeType, nodeAddress); + List deadServers = getDeadServers(); + if (!deadServers.contains(deadServerAddress)) { + deadServers.add(deadServerAddress); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); + } + } + + private void removeDeadServer(String nodeType, String nodeAddress) { + final String deadServerAddress = getDeadServerSuffix(nodeType, nodeAddress); + List deadServers = getDeadServers(); + if (deadServers.contains(deadServerAddress)) { + deadServers.remove(deadServerAddress); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); + } + } + + /** + * @return IP:Port->TimeMillis + */ + private Map getActiveWorkerServers() { + final String servers = readUtf8(kvStore.bGet(Constants.WORKER_TYPE)); + if (StringUtils.isEmpty(servers)) { + return new HashMap<>(); + } + return JSONUtils.toMap(servers, String.class, Long.class); + } + + private List getDeadServers() { + final String storedDeadServers = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS)); + if (StringUtils.isEmpty(storedDeadServers)) { + return new ArrayList<>(); + } + return new ArrayList<>(JSONUtils.toList(storedDeadServers, String.class)); + } + + /** + * @return IP:Port->TimeMillis + */ + private Map getActiveMasterServers() { + final String storedMasterServers = readUtf8(kvStore.bGet(Constants.MASTER_TYPE)); + if (StringUtils.isEmpty(storedMasterServers)) { + return new HashMap<>(); + } + return JSONUtils.toMap(storedMasterServers, String.class, Long.class); + } + + private String getDeadServerSuffix(String nodeType, String serverAddress) { + return nodeType + Constants.UNDERLINE + serverAddress; + } + + private void addWorkerGroup(String key) { + List workerGroupList = getWorkerGroups(); + String workerGroup = key.substring(key.indexOf(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS) + + Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS.length() + 1, key.lastIndexOf(Constants.SINGLE_SLASH)); + if (!workerGroupList.contains(workerGroup)) { + workerGroupList.add(workerGroup); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS, writeUtf8(JSONUtils.toJsonString(workerGroupList))); + } + } + + private void removeWorkerGroup(String nodeAddress) { + List workerGroupList = getWorkerGroups(); + final int originSize = workerGroupList.size(); + final Iterator iterator = workerGroupList.iterator(); + while (iterator.hasNext()) { + String group = iterator.next(); + final String groupKey = Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS + Constants.SINGLE_SLASH + group; + final String storedWorkerList = readUtf8(kvStore.bGet(groupKey)); + if (storedWorkerList != null) { + final List workers = JSONUtils.toList(storedWorkerList, String.class); + workers.remove(nodeAddress); + if (workers.isEmpty()) { + kvStore.bDelete(groupKey); + iterator.remove(); + } else { + kvStore.bPut(groupKey, writeUtf8(JSONUtils.toJsonString(workers))); + } + } + } + if (originSize != workerGroupList.size()) { + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS, writeUtf8(JSONUtils.toJsonString(workerGroupList))); + } + + } + + private List getWorkerGroups() { + final String storedWorkerGroup = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)); + if (StringUtils.isEmpty(storedWorkerGroup)) { + return new ArrayList<>(); + } + return new ArrayList<>(JSONUtils.toList(storedWorkerGroup, String.class)); + } + + private void addNodeData(String key, String value) { + Map nodeDataMap = getNodeDataMap(); + nodeDataMap.put(key, value); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_NODE, writeUtf8(JSONUtils.toJsonString(nodeDataMap))); + } + + private void removeNodeData(String key) { + Map nodeDataMap = getNodeDataMap(); + nodeDataMap.remove(key); + kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_NODE, writeUtf8(JSONUtils.toJsonString(nodeDataMap))); + } + + private Map getNodeDataMap() { + final String storedMasterServers = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_NODE)); + if (StringUtils.isEmpty(storedMasterServers)) { + return new HashMap<>(); + } + return JSONUtils.toMap(storedMasterServers, String.class, String.class); + } +} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java new file mode 100644 index 000000000000..99cd02172b50 --- /dev/null +++ b/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> 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 distributedLock = distributedLockMap.get(key); + if (distributedLock != null) { + distributedLock.unlock(); + } + distributedLockMap.remove(key); + ephemeralNodeManager.deleteHandler(key); + + } + + @Override + public Collection children(String key) { + final String result = readUtf8(kvStore.bGet(key)); + if (StringUtils.isEmpty(result)) { + return new ArrayList<>(); + } + final List 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 distributedLock = kvStore.getDistributedLock(key, properties.getDistributedLockTimeout().toMillis(), TimeUnit.MILLISECONDS); + final boolean lock = distributedLock.tryLock(); + if (lock) { + distributedLockMap.put(key, distributedLock); + } + return lock; + } + + @Override + public boolean releaseLock(String key) { + final DistributedLock distributedLock = distributedLockMap.get(key); + if (distributedLock != null) { + distributedLock.unlock(); + } + return true; + } + + @Override + public void close() { + ephemeralNodeManager.close(); + kvStore.shutdown(); + log.info("Closed raft registry..."); + } + +} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java new file mode 100644 index 000000000000..ccb0b2e9508f --- /dev/null +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java @@ -0,0 +1,47 @@ +/* + * 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 java.time.Duration; + +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.ConfigurationProperties; +import org.springframework.context.annotation.Configuration; + +import lombok.Data; + +@Data +@Configuration +@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "raft") +@ConfigurationProperties(prefix = "registry") +public class RaftRegistryProperties { + private String clusterName; + private String serverAddressList; + private String serverAddress; + private int serverPort; + private String logStorageDir; + private String dbStorageDir; + private Duration distributedLockTimeout = Duration.ofSeconds(3); + private Duration electionTimeout = Duration.ofMillis(1000); + private Duration snapshotInterval = Duration.ofSeconds(3600); + private Duration listenerCheckInterval = Duration.ofSeconds(2); + /** + * expireTime = listenerCheckInterval * connectionExpireFactor + */ + private int connectionExpireFactor = 2; +} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/pom.xml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/pom.xml index e81cc076eb48..f9e3d34750a2 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/pom.xml +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/pom.xml @@ -33,5 +33,6 @@ dolphinscheduler-registry-zookeeper dolphinscheduler-registry-mysql + dolphinscheduler-registry-raft From 0edeab5d4b5be7c84297fe8e8ad56d108864d08b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Tue, 26 Jul 2022 09:42:38 +0800 Subject: [PATCH 2/9] add license header in pom.xml --- .../dolphinscheduler-registry-raft/README.md | 5 +++-- .../dolphinscheduler-registry-raft/pom.xml | 17 ++++++++++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md index 962d544b0f55..69490e93e911 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md @@ -8,10 +8,11 @@ If you want to set the registry center as raft, you need to set the registry properties in master/worker/api's appplication.yml, -please remember change the server-port in appplication.yml. +worker and api address add `/learner` suffix in `server-address-list`, indicates that they are not participate in the leader election. -In PRO environment, worker and api add `/learner` suffix in `server-address-list` +`please remember change the server-port in appplication.yml`. +NOTE: In production environment, in order to achieve high availability, the master must be an odd number e.g 3 or 5. ```yaml registry: diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml index 1311870dfd2e..ffd38e1d5cd2 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml @@ -1,5 +1,20 @@ - + From 995afd351c47027bb44cf146fafa6ce264c71e67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Tue, 26 Jul 2022 15:03:09 +0800 Subject: [PATCH 3/9] add license --- dolphinscheduler-bom/pom.xml | 5 - dolphinscheduler-dist/release-docs/LICENSE | 5 + .../release-docs/licenses/LICENSE-bolt.txt | 201 ++++++++++++++++++ .../release-docs/licenses/LICENSE-hessian.txt | 201 ++++++++++++++++++ .../licenses/LICENSE-jraft-core.txt | 201 ++++++++++++++++++ .../licenses/LICENSE-sofa-common-tools.txt | 201 ++++++++++++++++++ .../dolphinscheduler-registry-raft/pom.xml | 4 - tools/dependencies/known-dependencies.txt | 9 + 8 files changed, 818 insertions(+), 9 deletions(-) create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-bolt.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-hessian.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-jraft-core.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-sofa-common-tools.txt diff --git a/dolphinscheduler-bom/pom.xml b/dolphinscheduler-bom/pom.xml index b14a1aa1b4b2..45a2cb820275 100644 --- a/dolphinscheduler-bom/pom.xml +++ b/dolphinscheduler-bom/pom.xml @@ -595,11 +595,6 @@ ${json-path.version} - - com.alipay.sofa - jraft-core - ${jraft.version} - com.alipay.sofa jraft-rheakv-core diff --git a/dolphinscheduler-dist/release-docs/LICENSE b/dolphinscheduler-dist/release-docs/LICENSE index 52c77584550f..2ada5826a388 100644 --- a/dolphinscheduler-dist/release-docs/LICENSE +++ b/dolphinscheduler-dist/release-docs/LICENSE @@ -447,6 +447,11 @@ The text of each license is also included at licenses/LICENSE-[project].txt. kotlin 1.6.21: https://mvnrepository.com/artifact/org.jetbrains.kotlin/kotlin-stdlib, Apache 2.0 JetBrains annotations: https://mvnrepository.com/artifact/org.jetbrains/annotations, Apache 2.0 + jraft-core-1.3.11.jar: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-core/1.3.11 + jraft-rheakv-core-1.3.11.jar: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-rheakv-core/1.3.11 + sofa-common-tools-1.0.12.jar: https://mvnrepository.com/artifact/com.alipay.sofa.common/sofa-common-tools/1.0.12 + bolt-1.6.4.jar: https://mvnrepository.com/artifact/com.alipay.sofa/bolt/1.6.4 + hessian-3.3.6.jar: https://mvnrepository.com/artifact/com.alipay.sofa/hessian/3.3.6 ======================================================================== BSD licenses ======================================================================== diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-bolt.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-bolt.txt new file mode 100644 index 000000000000..b0ff39019adf --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-bolt.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2018 Ant Financial Services Group Co., Ltd. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-hessian.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-hessian.txt new file mode 100644 index 000000000000..26c62dba89ce --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-hessian.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2018 Ant Small and Micro Financial Services Group Co., Ltd. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-jraft-core.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-jraft-core.txt new file mode 100644 index 000000000000..b0ff39019adf --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-jraft-core.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2018 Ant Financial Services Group Co., Ltd. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-sofa-common-tools.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-sofa-common-tools.txt new file mode 100644 index 000000000000..f49a4e16e68b --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-sofa-common-tools.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml index ffd38e1d5cd2..f1b9b0828856 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/pom.xml @@ -35,10 +35,6 @@ org.apache.dolphinscheduler dolphinscheduler-common - - com.alipay.sofa - jraft-core - com.alipay.sofa jraft-rheakv-core diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index b28957fd7a6a..84a64507f41f 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -289,3 +289,12 @@ zeppelin-client-0.10.1.jar zeppelin-common-0.10.1.jar zjsonpatch-0.3.0.jar zookeeper-3.4.14.jar +json-path-2.7.0.jar +accessors-smart-2.4.7.jar +asm-9.1.jar +json-smart-2.4.7.jar +jraft-core-1.3.11.jar +jraft-rheakv-core-1.3.11.jar +sofa-common-tools-1.0.12.jar +bolt-1.6.4.jar +hessian-3.3.6.jar From fa59e4b697e7e300b581838cd7057eb93fbd823f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Tue, 26 Jul 2022 15:34:44 +0800 Subject: [PATCH 4/9] refactor addDeadServer method --- .../plugin/registry/raft/EphemeralNodeManager.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java index abb9aec1c3ce..8e805f46871b 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java @@ -139,7 +139,7 @@ private void checkActiveNode() { final String nodeAddress = entry.getKey(); activeMasterServers.remove(nodeAddress); updateActiveMaster(activeMasterServers); - addDeadServer(Constants.MASTER_TYPE, nodeAddress); + addDeadServer(getDeadServerSuffix(Constants.MASTER_TYPE, nodeAddress)); if (nodeAddress.split(Constants.COLON)[0].equals(NetUtils.getHost())) { connectionState = ConnectionState.DISCONNECTED; triggerListener(ConnectionState.DISCONNECTED); @@ -156,7 +156,7 @@ private void checkActiveNode() { activeWorkerServers.remove(nodeAddress); updateActiveWorker(nodeAddress, activeWorkerServers); removeWorkerGroup(nodeAddress); - addDeadServer(Constants.WORKER_TYPE, nodeAddress); + addDeadServer(getDeadServerSuffix(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); @@ -249,11 +249,7 @@ public void putHandler(String key, String value) { removeDeadServer(Constants.WORKER_TYPE, nodeAddress); addNodeData(key, value); } else if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS)) { - final List deadServers = getDeadServers(); - if (!deadServers.contains(nodeAddress)) { - deadServers.add(nodeAddress); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); - } + addDeadServer(nodeAddress); } } @@ -288,8 +284,7 @@ private void updateActiveWorker(String key, Map activeNodes) { kvStore.bPut(key.substring(0, key.lastIndexOf(Constants.SINGLE_SLASH)), writeUtf8(JSONUtils.toJsonString(activeNodes.keySet()))); } - private void addDeadServer(String nodeType, String nodeAddress) { - final String deadServerAddress = getDeadServerSuffix(nodeType, nodeAddress); + private void addDeadServer(String deadServerAddress) { List deadServers = getDeadServers(); if (!deadServers.contains(deadServerAddress)) { deadServers.add(deadServerAddress); From 9b75db729e1cbddf71f01d7c479cbce376aedf76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Sun, 31 Jul 2022 09:13:18 +0800 Subject: [PATCH 5/9] add LICENSE --- dolphinscheduler-dist/release-docs/LICENSE | 18 +- .../licenses/LICENSE-affinity.txt | 201 +++++++++++++++++ .../licenses/LICENSE-annotations.txt | 202 ++++++++++++++++++ .../licenses/LICENSE-disruptor.txt | 202 ++++++++++++++++++ .../licenses/LICENSE-jctools-core.txt | 202 ++++++++++++++++++ .../licenses/LICENSE-metrics-core.txt | 202 ++++++++++++++++++ .../licenses/LICENSE-rocksdbjni.txt | 202 ++++++++++++++++++ tools/dependencies/known-dependencies.txt | 8 + 8 files changed, 1231 insertions(+), 6 deletions(-) create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-affinity.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-annotations.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-disruptor.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-jctools-core.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-metrics-core.txt create mode 100644 dolphinscheduler-dist/release-docs/licenses/LICENSE-rocksdbjni.txt diff --git a/dolphinscheduler-dist/release-docs/LICENSE b/dolphinscheduler-dist/release-docs/LICENSE index 2ada5826a388..f4d20af7e347 100644 --- a/dolphinscheduler-dist/release-docs/LICENSE +++ b/dolphinscheduler-dist/release-docs/LICENSE @@ -447,11 +447,17 @@ The text of each license is also included at licenses/LICENSE-[project].txt. kotlin 1.6.21: https://mvnrepository.com/artifact/org.jetbrains.kotlin/kotlin-stdlib, Apache 2.0 JetBrains annotations: https://mvnrepository.com/artifact/org.jetbrains/annotations, Apache 2.0 - jraft-core-1.3.11.jar: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-core/1.3.11 - jraft-rheakv-core-1.3.11.jar: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-rheakv-core/1.3.11 - sofa-common-tools-1.0.12.jar: https://mvnrepository.com/artifact/com.alipay.sofa.common/sofa-common-tools/1.0.12 - bolt-1.6.4.jar: https://mvnrepository.com/artifact/com.alipay.sofa/bolt/1.6.4 - hessian-3.3.6.jar: https://mvnrepository.com/artifact/com.alipay.sofa/hessian/3.3.6 + jraft-core 1.3.11: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-core/1.3.11, Apache 2.0 + jraft-rheakv-core 1.3.11: https://mvnrepository.com/artifact/com.alipay.sofa/jraft-rheakv-core/1.3.11, Apache 2.0 + sofa-common-tools 1.0.12: https://mvnrepository.com/artifact/com.alipay.sofa.common/sofa-common-tools/1.0.12, Apache 2.0 + bolt 1.6.4: https://mvnrepository.com/artifact/com.alipay.sofa/bolt/1.6.4, Apache 2.0 + hessian 3.3.6: https://mvnrepository.com/artifact/com.alipay.sofa/hessian/3.3.6, Apache 2.0 + affinity 3.1.7: https://mvnrepository.com/artifact/net.openhft/affinity/3.1.7, Apache 2.0 + annotations 12.0: https://mvnrepository.com/artifact/com.intellij/annotations/12.0, Apache 2.0 + disruptor 3.3.7: https://mvnrepository.com/artifact/com.lmax/disruptor/3.3.7, Apache 2.0 + jctools-core 2.1.1: https://mvnrepository.com/artifact/org.jctools/jctools-core/2.1.1, Apache 2.0 + metrics-core 4.1.26: https://mvnrepository.com/artifact/io.dropwizard.metrics/metrics-core/4.1.26, Apache 2.0 + rocksdbjni 6.22.1.1: https://search.maven.org/artifact/org.rocksdb/rocksdbjni/6.22.1.1, Apache 2.0 ======================================================================== BSD licenses ======================================================================== @@ -466,7 +472,6 @@ The text of each license is also included at licenses/LICENSE-[project].txt. jline 0.9.94: https://github.com/jline/jline3, BSD jsch 0.1.42: https://mvnrepository.com/artifact/com.jcraft/jsch/0.1.42, BSD postgresql 42.3.4: https://mvnrepository.com/artifact/org.postgresql/postgresql/42.3.4, BSD 2-clause - protobuf-java 2.5.0: https://mvnrepository.com/artifact/com.google.protobuf/protobuf-java/2.5.0, BSD 2-clause paranamer 2.3: https://mvnrepository.com/artifact/com.thoughtworks.paranamer/paranamer/2.3, BSD threetenbp 1.3.6: https://mvnrepository.com/artifact/org.threeten/threetenbp/1.3.6, BSD 3-clause xmlenc 0.52: https://mvnrepository.com/artifact/xmlenc/xmlenc/0.52, BSD @@ -475,6 +480,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. janino 3.1.7: https://mvnrepository.com/artifact/org.codehaus.janino/janino/3.1.7, BSD 3-clause commons-compiler 3.1.7: https://mvnrepository.com/artifact/org.codehaus.janino/janino/3.1.7, BSD 3-clause automaton 1.11-8 https://mvnrepository.com/artifact/dk.brics.automaton/automaton/1.11-8, BSD 2-clause + protobuf-java 3.5.1.jar: https://mvnrepository.com/artifact/com.google.protobuf/protobuf-java/3.5.1, BSD 3-clause ======================================================================== CDDL licenses diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-affinity.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-affinity.txt new file mode 100644 index 000000000000..aa29d40653a8 --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-affinity.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2014-2018 Chronicle Software Ltd + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-annotations.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-annotations.txt new file mode 100644 index 000000000000..8541f77fc73d --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-annotations.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2000-2021 JetBrains s.r.o. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-disruptor.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-disruptor.txt new file mode 100644 index 000000000000..8541f77fc73d --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-disruptor.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2000-2021 JetBrains s.r.o. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-jctools-core.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-jctools-core.txt new file mode 100644 index 000000000000..8541f77fc73d --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-jctools-core.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2000-2021 JetBrains s.r.o. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-metrics-core.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-metrics-core.txt new file mode 100644 index 000000000000..8541f77fc73d --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-metrics-core.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2000-2021 JetBrains s.r.o. + + Licensed 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. \ No newline at end of file diff --git a/dolphinscheduler-dist/release-docs/licenses/LICENSE-rocksdbjni.txt b/dolphinscheduler-dist/release-docs/licenses/LICENSE-rocksdbjni.txt new file mode 100644 index 000000000000..8541f77fc73d --- /dev/null +++ b/dolphinscheduler-dist/release-docs/licenses/LICENSE-rocksdbjni.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2000-2021 JetBrains s.r.o. + + Licensed 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. \ No newline at end of file diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 84a64507f41f..db4a0b732d82 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -298,3 +298,11 @@ jraft-rheakv-core-1.3.11.jar sofa-common-tools-1.0.12.jar bolt-1.6.4.jar hessian-3.3.6.jar +affinity-3.1.7.jar +annotations-12.0.jar +asm-6.0.jar +disruptor-3.3.7.jar +jctools-core-2.1.1.jar +metrics-core-4.1.26.jar +protobuf-java-3.5.1.jar +rocksdbjni-6.22.1.1.jar From 74d9c9a4d628251227438a737df4248aeafdde4f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Sun, 31 Jul 2022 23:36:58 +0800 Subject: [PATCH 6/9] improve raft cluster --- .../dolphinscheduler-registry-raft/README.md | 9 +- .../registry/raft/EphemeralNodeManager.java | 400 ------------------ .../raft/RaftConnectionStateListener.java | 54 +++ .../plugin/registry/raft/RaftRegistry.java | 107 +++-- .../registry/raft/RaftRegistryProperties.java | 9 +- .../raft/SubscribeListenerManager.java | 162 +++++++ 6 files changed, 301 insertions(+), 440 deletions(-) delete mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java create mode 100644 dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md index 69490e93e911..87fac8415147 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/README.md @@ -20,13 +20,14 @@ registry: cluster-name: dolphinscheduler server-address-list: 127.0.0.1:8181,127.0.0.1:8182/learner,127.0.0.1:8183/learner log-storage-dir: raft-data/ - db-storage-dir: raft-db/ election-timeout: 1000ms - snapshot-interval: 1800s listener-check-interval: 2s - connection-expire-factor: 2 + distributed-lock-timeout: 3s server-address: 127.0.0.1 - server-port: 8181 + server-port: 8183 + module: api + rpc-core-threads: 8 + rpc-timeout-millis: 5000ms ``` Then you can start your DolphinScheduler cluster, your cluster will use raft cluster as registry center to diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java deleted file mode 100644 index 8e805f46871b..000000000000 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/EphemeralNodeManager.java +++ /dev/null @@ -1,400 +0,0 @@ -/* - * 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 activeMasterServers - * 3. maintain Map activeWorkerServers - * 4. maintain Map> master servers - * 5. maintain Map> worker servers - * 6. maintain Map> dead servers - */ -@Slf4j -public class EphemeralNodeManager implements AutoCloseable { - private final List connectionListeners = Collections.synchronizedList(new ArrayList<>()); - - private final Map> 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 activeMasterServers = getActiveMasterServers(); - for (Map.Entry entry : activeMasterServers.entrySet()) { - if ((System.currentTimeMillis() - entry.getValue()) > expireTime) { - final String nodeAddress = entry.getKey(); - activeMasterServers.remove(nodeAddress); - updateActiveMaster(activeMasterServers); - addDeadServer(getDeadServerSuffix(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 activeWorkerServers = getActiveWorkerServers(); - for (Map.Entry entry : activeWorkerServers.entrySet()) { - if ((System.currentTimeMillis() - entry.getValue()) > expireTime) { - final String nodeAddress = entry.getKey(); - activeWorkerServers.remove(nodeAddress); - updateActiveWorker(nodeAddress, activeWorkerServers); - removeWorkerGroup(nodeAddress); - addDeadServer(getDeadServerSuffix(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 nodeDataMap = new ConcurrentHashMap<>(); - - @Override - public void run() { - subscribeCheck(); - } - - private void subscribeCheck() { - if (dataSubScribeMap.isEmpty()) { - return; - } - final Map currentNodeDataMap = getNodeDataMap(); - // find the different - Map addedData = new HashMap<>(); - Map deletedData = new HashMap<>(); - Map updatedData = new HashMap<>(); - for (Map.Entry 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()); - HeartBeat oldHeartBeat = HeartBeat.decodeHeartBeat(oldData); - if (newHeartBeat != null && newHeartBeat.getReportTime() != oldHeartBeat.getReportTime()) { - updatedData.put(entry.getKey(), entry.getValue()); - } - } - } - for (Map.Entry entry : nodeDataMap.entrySet()) { - if (!currentNodeDataMap.containsKey(entry.getKey())) { - deletedData.put(entry.getKey(), entry.getValue()); - } - } - nodeDataMap.clear(); - nodeDataMap.putAll(currentNodeDataMap); - // trigger listener - for (Map.Entry> entry : dataSubScribeMap.entrySet()) { - String subscribeKey = entry.getKey(); - List subscribeListeners = entry.getValue(); - triggerListener(addedData, subscribeKey, subscribeListeners, Event.Type.ADD); - triggerListener(deletedData, subscribeKey, subscribeListeners, Event.Type.REMOVE); - triggerListener(updatedData, subscribeKey, subscribeListeners, Event.Type.UPDATE); - } - - } - - private void triggerListener(Map nodeDataMap, String subscribeKey, List subscribeListeners, Event.Type type) { - for (Map.Entry entry : nodeDataMap.entrySet()) { - final String key = entry.getKey(); - if (key.startsWith(subscribeKey)) { - subscribeListeners.forEach(listener -> listener.notify(new Event(key, key, entry.getValue(), type))); - } - } - } - } - - public static String convertTimeToString(Long time) { - DateTimeFormatter ftf = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"); - return ftf.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(time), ZoneId.systemDefault())); - } - - public void putHandler(String key, String value) { - final String nodeAddress = key.substring(key.lastIndexOf(Constants.SINGLE_SLASH) + 1); - //update heart beat time and node set - if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS)) { - Map activeMasterServers = getActiveMasterServers(); - activeMasterServers.put(nodeAddress, System.currentTimeMillis()); - updateActiveMaster(activeMasterServers); - removeDeadServer(Constants.MASTER_TYPE, nodeAddress); - addNodeData(key, value); - } else if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)) { - Map activeWorkerServers = getActiveWorkerServers(); - activeWorkerServers.put(nodeAddress, System.currentTimeMillis()); - updateActiveWorker(key, activeWorkerServers); - addWorkerGroup(key); - removeDeadServer(Constants.WORKER_TYPE, nodeAddress); - addNodeData(key, value); - } else if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS)) { - addDeadServer(nodeAddress); - } - - } - - public void deleteHandler(String key) { - final String nodeAddress = key.substring(key.lastIndexOf(Constants.SINGLE_SLASH) + 1); - if (key.contains(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS)) { - Map activeMasterServers = getActiveMasterServers(); - activeMasterServers.remove(nodeAddress); - updateActiveMaster(activeMasterServers); - removeNodeData(nodeAddress); - log.info("Raft registry remove master server {}", nodeAddress); - } else if (key.contains(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)) { - Map activeWorkerServers = getActiveWorkerServers(); - activeWorkerServers.remove(nodeAddress); - updateActiveWorker(key, activeWorkerServers); - removeWorkerGroup(nodeAddress); - removeNodeData(nodeAddress); - log.info("Raft registry remove worker server {}", nodeAddress); - } - } - - private void updateActiveMaster(Map activeNodes) { - kvStore.bPut(Constants.MASTER_TYPE, writeUtf8(JSONUtils.toJsonString(activeNodes))); - //Update the mapping of the master group and master node list - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_MASTERS, writeUtf8(JSONUtils.toJsonString(activeNodes.keySet()))); - } - - private void updateActiveWorker(String key, Map activeNodes) { - kvStore.bPut(Constants.WORKER_TYPE, writeUtf8(JSONUtils.toJsonString(activeNodes))); - //Update the mapping of the worker group and worker node list - kvStore.bPut(key.substring(0, key.lastIndexOf(Constants.SINGLE_SLASH)), writeUtf8(JSONUtils.toJsonString(activeNodes.keySet()))); - } - - private void addDeadServer(String deadServerAddress) { - List deadServers = getDeadServers(); - if (!deadServers.contains(deadServerAddress)) { - deadServers.add(deadServerAddress); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); - } - } - - private void removeDeadServer(String nodeType, String nodeAddress) { - final String deadServerAddress = getDeadServerSuffix(nodeType, nodeAddress); - List deadServers = getDeadServers(); - if (deadServers.contains(deadServerAddress)) { - deadServers.remove(deadServerAddress); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS, writeUtf8(JSONUtils.toJsonString(deadServers))); - } - } - - /** - * @return IP:Port->TimeMillis - */ - private Map getActiveWorkerServers() { - final String servers = readUtf8(kvStore.bGet(Constants.WORKER_TYPE)); - if (StringUtils.isEmpty(servers)) { - return new HashMap<>(); - } - return JSONUtils.toMap(servers, String.class, Long.class); - } - - private List getDeadServers() { - final String storedDeadServers = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_DEAD_SERVERS)); - if (StringUtils.isEmpty(storedDeadServers)) { - return new ArrayList<>(); - } - return new ArrayList<>(JSONUtils.toList(storedDeadServers, String.class)); - } - - /** - * @return IP:Port->TimeMillis - */ - private Map getActiveMasterServers() { - final String storedMasterServers = readUtf8(kvStore.bGet(Constants.MASTER_TYPE)); - if (StringUtils.isEmpty(storedMasterServers)) { - return new HashMap<>(); - } - return JSONUtils.toMap(storedMasterServers, String.class, Long.class); - } - - private String getDeadServerSuffix(String nodeType, String serverAddress) { - return nodeType + Constants.UNDERLINE + serverAddress; - } - - private void addWorkerGroup(String key) { - List workerGroupList = getWorkerGroups(); - String workerGroup = key.substring(key.indexOf(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS) - + Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS.length() + 1, key.lastIndexOf(Constants.SINGLE_SLASH)); - if (!workerGroupList.contains(workerGroup)) { - workerGroupList.add(workerGroup); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS, writeUtf8(JSONUtils.toJsonString(workerGroupList))); - } - } - - private void removeWorkerGroup(String nodeAddress) { - List workerGroupList = getWorkerGroups(); - final int originSize = workerGroupList.size(); - final Iterator iterator = workerGroupList.iterator(); - while (iterator.hasNext()) { - String group = iterator.next(); - final String groupKey = Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS + Constants.SINGLE_SLASH + group; - final String storedWorkerList = readUtf8(kvStore.bGet(groupKey)); - if (storedWorkerList != null) { - final List workers = JSONUtils.toList(storedWorkerList, String.class); - workers.remove(nodeAddress); - if (workers.isEmpty()) { - kvStore.bDelete(groupKey); - iterator.remove(); - } else { - kvStore.bPut(groupKey, writeUtf8(JSONUtils.toJsonString(workers))); - } - } - } - if (originSize != workerGroupList.size()) { - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS, writeUtf8(JSONUtils.toJsonString(workerGroupList))); - } - - } - - private List getWorkerGroups() { - final String storedWorkerGroup = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)); - if (StringUtils.isEmpty(storedWorkerGroup)) { - return new ArrayList<>(); - } - return new ArrayList<>(JSONUtils.toList(storedWorkerGroup, String.class)); - } - - private void addNodeData(String key, String value) { - Map nodeDataMap = getNodeDataMap(); - nodeDataMap.put(key, value); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_NODE, writeUtf8(JSONUtils.toJsonString(nodeDataMap))); - } - - private void removeNodeData(String key) { - Map nodeDataMap = getNodeDataMap(); - nodeDataMap.remove(key); - kvStore.bPut(Constants.REGISTRY_DOLPHINSCHEDULER_NODE, writeUtf8(JSONUtils.toJsonString(nodeDataMap))); - } - - private Map getNodeDataMap() { - final String storedMasterServers = readUtf8(kvStore.bGet(Constants.REGISTRY_DOLPHINSCHEDULER_NODE)); - if (StringUtils.isEmpty(storedMasterServers)) { - return new HashMap<>(); - } - return JSONUtils.toMap(storedMasterServers, String.class, String.class); - } -} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java new file mode 100644 index 000000000000..d7ddfbad1906 --- /dev/null +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java @@ -0,0 +1,54 @@ +package org.apache.dolphinscheduler.plugin.registry.raft; + +import org.apache.dolphinscheduler.registry.api.ConnectionListener; +import org.apache.dolphinscheduler.registry.api.ConnectionState; + +import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.core.Replicator; +import com.alipay.sofa.jraft.entity.PeerId; + +public class RaftConnectionStateListener implements Replicator.ReplicatorStateListener { + private final ConnectionListener connectionListener; + private ConnectionState connectionState; + + public RaftConnectionStateListener(ConnectionListener connectionListener) { + this.connectionListener = connectionListener; + } + + @Override + public void onCreated(PeerId peerId) { + + } + + @Override + public void onError(PeerId peerId, Status status) { + + } + + @Override + public void onDestroyed(PeerId peerId) { + + } + + @Override + public void stateChanged(PeerId peer, ReplicatorState newState) { + switch (newState) { + case CREATED: + connectionState = ConnectionState.CONNECTED; + break; + case ONLINE: + if (connectionState == ConnectionState.DISCONNECTED || connectionState == ConnectionState.SUSPENDED) { + connectionState = ConnectionState.RECONNECTED; + } + break; + case OFFLINE: + connectionState = ConnectionState.SUSPENDED; + break; + case DESTROYED: + connectionState = ConnectionState.DISCONNECTED; + break; + default: + } + connectionListener.onUpdate(connectionState); + } +} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java index 99cd02172b50..161bdd55ce24 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java @@ -20,12 +20,15 @@ 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.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 org.apache.commons.lang3.RandomStringUtils; + import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; @@ -34,21 +37,23 @@ 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.Node; +import com.alipay.sofa.jraft.NodeManager; +import com.alipay.sofa.jraft.entity.PeerId; 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.RpcOptions; 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.KVEntry; import com.alipay.sofa.jraft.rhea.storage.StorageType; import com.alipay.sofa.jraft.rhea.util.concurrent.DistributedLock; import com.alipay.sofa.jraft.util.Endpoint; @@ -62,11 +67,17 @@ public class RaftRegistry implements Registry { private final Map> distributedLockMap = new ConcurrentHashMap<>(); - private RheaKVStore kvStore; + private final RheaKVStore kvStore; + + private final RaftRegistryProperties properties; + + private SubscribeListenerManager subscribeListenerManager; + + private static final String REGISTRY_DOLPHINSCHEDULER_WORKER_GROUPS = "/nodes/worker-groups"; - private RaftRegistryProperties properties; + private static final String RANDOM_STRING = RandomStringUtils.randomAlphanumeric(200); - private EphemeralNodeManager ephemeralNodeManager; + private static final String API_TYPE = "api"; public RaftRegistry(RaftRegistryProperties properties) { this.properties = properties; @@ -76,45 +87,49 @@ public RaftRegistry(RaftRegistryProperties properties) { .config(); NodeOptions nodeOptions = new NodeOptions(); nodeOptions.setElectionTimeoutMs((int) properties.getElectionTimeout().toMillis()); - nodeOptions.setSnapshotIntervalSecs((int) properties.getSnapshotInterval().getSeconds()); + final Endpoint serverAddress = new Endpoint(properties.getServerAddress(), properties.getServerPort()); final StoreEngineOptions storeOpts = StoreEngineOptionsConfigured.newConfigured() - .withStorageType(StorageType.RocksDB) - .withRocksDBOptions(RocksDBOptionsConfigured.newConfigured().withDbPath(properties.getDbStorageDir()).config()) + .withStorageType(StorageType.Memory) .withRaftDataPath(properties.getLogStorageDir()) - .withServerAddress(new Endpoint(properties.getServerAddress(), properties.getServerPort())) + .withServerAddress(serverAddress) .withCommonNodeOptions(nodeOptions) + .withKvRpcCoreThreads(properties.getRpcCoreThreads()) .config(); + RpcOptions rpcOptions = new RpcOptions(); + rpcOptions.setCallbackExecutorCorePoolSize(properties.getRpcCoreThreads()); + rpcOptions.setRpcTimeoutMillis((int) properties.getRpcTimeoutMillis().toMillis()); final RheaKVStoreOptions opts = RheaKVStoreOptionsConfigured.newConfigured() .withClusterName(properties.getClusterName()) .withUseParallelCompress(true) .withInitialServerList(properties.getServerAddressList()) .withStoreEngineOptions(storeOpts) .withPlacementDriverOptions(pdOpts) + .withRpcOptions(rpcOptions) .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(); + if (!properties.getModule().equalsIgnoreCase(API_TYPE)) { + this.subscribeListenerManager = new SubscribeListenerManager(properties, kvStore); + subscribeListenerManager.start(); + } } @Override public boolean subscribe(String path, SubscribeListener listener) { - return ephemeralNodeManager.addSubscribeListener(path, listener); + return subscribeListenerManager.addSubscribeListener(path, listener); } @Override public void unsubscribe(String path) { - ephemeralNodeManager.removeSubscribeListener(path); + subscribeListenerManager.removeSubscribeListener(path); } @Override public void addConnectionStateListener(ConnectionListener listener) { - ephemeralNodeManager.addConnectionListener(listener); + final String groupId = properties.getClusterName() + "--1"; + final Node node = NodeManager.getInstance().get(groupId, new PeerId((properties.getServerAddress()), properties.getServerPort())); + node.addReplicatorStateListener(new RaftConnectionStateListener(listener)); } @Override @@ -124,11 +139,28 @@ public String get(String key) { @Override public void put(String key, String value, boolean deleteOnDisconnect) { - if (StringUtils.isBlank(value)) { - return; + kvStore.bPut(key, writeUtf8(value)); + if (key.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS + Constants.SINGLE_SLASH)) { + addWorkerGroup(key); } - readUtf8(kvStore.bGetAndPut(key, writeUtf8(value))); - ephemeralNodeManager.putHandler(key, value); + } + + private void addWorkerGroup(String key) { + List workerGroupList = getWorkerGroups(); + String workerGroup = key.substring(key.indexOf(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS) + + Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS.length() + 1, key.lastIndexOf(Constants.SINGLE_SLASH)); + if (!workerGroupList.contains(workerGroup)) { + workerGroupList.add(workerGroup); + kvStore.bPut(REGISTRY_DOLPHINSCHEDULER_WORKER_GROUPS, writeUtf8(JSONUtils.toJsonString(workerGroupList))); + } + } + + private List getWorkerGroups() { + final String storedWorkerGroup = readUtf8(kvStore.bGet(REGISTRY_DOLPHINSCHEDULER_WORKER_GROUPS)); + if (StringUtils.isEmpty(storedWorkerGroup)) { + return new ArrayList<>(); + } + return new ArrayList<>(JSONUtils.toList(storedWorkerGroup, String.class)); } @Override @@ -139,17 +171,32 @@ public void delete(String key) { distributedLock.unlock(); } distributedLockMap.remove(key); - ephemeralNodeManager.deleteHandler(key); - } @Override public Collection children(String key) { - final String result = readUtf8(kvStore.bGet(key)); - if (StringUtils.isEmpty(result)) { - return new ArrayList<>(); + List children = new ArrayList<>(); + if (key.equals(Constants.REGISTRY_DOLPHINSCHEDULER_WORKERS)) { + //get all worker groups + children = getWorkerGroups(); + + } else { + final List result = kvStore.bScan(key, key + Constants.SINGLE_SLASH + RANDOM_STRING); + if (result.isEmpty()) { + return new ArrayList<>(); + } + for (final KVEntry kv : result) { + if (StringUtils.isEmpty(readUtf8(kv.getValue()))) { + continue; + } + final String entryKey = readUtf8(kv.getKey()); + if (StringUtils.isEmpty(entryKey)) { + continue; + } + String child = entryKey.substring(entryKey.lastIndexOf(Constants.SINGLE_SLASH) + 1); + children.add(child); + } } - final List children = JSONUtils.toList(result, String.class); children.sort(Comparator.reverseOrder()); return children; } @@ -180,7 +227,7 @@ public boolean releaseLock(String key) { @Override public void close() { - ephemeralNodeManager.close(); + subscribeListenerManager.close(); kvStore.shutdown(); log.info("Closed raft registry..."); } diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java index ccb0b2e9508f..d0e213f4d38f 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistryProperties.java @@ -35,13 +35,10 @@ public class RaftRegistryProperties { private String serverAddress; private int serverPort; private String logStorageDir; - private String dbStorageDir; private Duration distributedLockTimeout = Duration.ofSeconds(3); private Duration electionTimeout = Duration.ofMillis(1000); - private Duration snapshotInterval = Duration.ofSeconds(3600); private Duration listenerCheckInterval = Duration.ofSeconds(2); - /** - * expireTime = listenerCheckInterval * connectionExpireFactor - */ - private int connectionExpireFactor = 2; + private String module = "master"; + private int rpcCoreThreads = 8; + private Duration rpcTimeoutMillis = Duration.ofMillis(5000); } diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java new file mode 100644 index 000000000000..08cd63ca57ee --- /dev/null +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java @@ -0,0 +1,162 @@ +/* + * 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 org.apache.dolphinscheduler.common.Constants; +import org.apache.dolphinscheduler.common.utils.HeartBeat; +import org.apache.dolphinscheduler.registry.api.Event; +import org.apache.dolphinscheduler.registry.api.SubscribeListener; +import org.apache.dolphinscheduler.spi.utils.StringUtils; + +import java.util.ArrayList; +import java.util.HashMap; +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 java.util.regex.Pattern; + +import com.alipay.sofa.jraft.rhea.client.RheaKVStore; +import com.alipay.sofa.jraft.rhea.storage.KVEntry; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +public class SubscribeListenerManager implements AutoCloseable { + + private final Map> dataSubScribeMap = new ConcurrentHashMap<>(); + + private final RaftRegistryProperties properties; + + private final RheaKVStore kvStore; + + private static final String REGEX = "\\d{1,3}(\\.\\d{1,3}){3,5}:\\d+"; + + public SubscribeListenerManager(RaftRegistryProperties properties, RheaKVStore kvStore) { + this.properties = properties; + this.kvStore = kvStore; + } + + private final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool( + 1, + new ThreadFactoryBuilder().setNameFormat("SubscribeListenerCheckThread").setDaemon(true).build()); + + public void start() { + scheduledExecutorService.scheduleWithFixedDelay(new SubscribeCheckTask(), + properties.getListenerCheckInterval().toMillis(), + properties.getListenerCheckInterval().toMillis(), + TimeUnit.MILLISECONDS); + } + + @Override + public void close() { + dataSubScribeMap.clear(); + scheduledExecutorService.shutdown(); + } + + 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 SubscribeCheckTask implements Runnable { + + private final Map nodeDataMap = new ConcurrentHashMap<>(); + + @Override + public void run() { + if (dataSubScribeMap.isEmpty()) { + return; + } + final Map currentNodeDataMap = getNodeDataMap(); + if (currentNodeDataMap.isEmpty()) { + return; + } + // find the different + Map addedData = new HashMap<>(); + Map deletedData = new HashMap<>(); + Map updatedData = new HashMap<>(); + for (Map.Entry 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()); + HeartBeat oldHeartBeat = HeartBeat.decodeHeartBeat(oldData); + if (newHeartBeat != null && newHeartBeat.getReportTime() != oldHeartBeat.getReportTime()) { + updatedData.put(entry.getKey(), entry.getValue()); + } + } + } + for (Map.Entry entry : nodeDataMap.entrySet()) { + if (!currentNodeDataMap.containsKey(entry.getKey())) { + deletedData.put(entry.getKey(), entry.getValue()); + } + } + nodeDataMap.clear(); + nodeDataMap.putAll(currentNodeDataMap); + // trigger listener + for (Map.Entry> entry : dataSubScribeMap.entrySet()) { + String subscribeKey = entry.getKey(); + List subscribeListeners = entry.getValue(); + if (!addedData.isEmpty()) { + triggerListener(addedData, subscribeKey, subscribeListeners, Event.Type.ADD); + } + if (!updatedData.isEmpty()) { + triggerListener(updatedData, subscribeKey, subscribeListeners, Event.Type.UPDATE); + } + if (!deletedData.isEmpty()) { + triggerListener(deletedData, subscribeKey, subscribeListeners, Event.Type.REMOVE); + } + } + } + + /** + * get node path and heartbeat map + * @return + */ + private Map getNodeDataMap() { + Pattern pattern = Pattern.compile(REGEX); + HashMap nodeDataMap = new HashMap<>(); + final List entryList = kvStore.bScan(Constants.REGISTRY_DOLPHINSCHEDULER_NODE + Constants.SINGLE_SLASH, null); + for (KVEntry kvEntry : entryList) { + final String value = readUtf8(kvEntry.getValue()); + final String entryKey = readUtf8(kvEntry.getKey()); + if (StringUtils.isEmpty(value) || pattern.matcher(value).find() || !entryKey.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_NODE)) { + continue; + } + nodeDataMap.put(entryKey, value); + } + return nodeDataMap; + } + + private void triggerListener(Map nodeDataMap, String subscribeKey, List subscribeListeners, Event.Type type) { + for (Map.Entry entry : nodeDataMap.entrySet()) { + final String key = entry.getKey(); + if (key.startsWith(subscribeKey)) { + subscribeListeners.forEach(listener -> listener.notify(new Event(key, key, entry.getValue(), type))); + } + } + } + } +} From f43e4f83cc0add8ba85cec18f6b345fb17c2f7b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Sun, 31 Jul 2022 23:43:36 +0800 Subject: [PATCH 7/9] add license header --- .../raft/RaftConnectionStateListener.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java index d7ddfbad1906..8ad258594f34 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java @@ -1,3 +1,20 @@ +/* + * 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 org.apache.dolphinscheduler.registry.api.ConnectionListener; From 3f4e71a9088a5352c276f44bf0010d5e738d8db8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Mon, 1 Aug 2022 22:11:48 +0800 Subject: [PATCH 8/9] fix code smells --- .../dolphinscheduler/common/Constants.java | 5 ++++ .../raft/RaftConnectionStateListener.java | 9 +++++--- .../plugin/registry/raft/RaftRegistry.java | 11 +++------ .../raft/SubscribeListenerManager.java | 23 ++++++++----------- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java index 635076120534..03497fde7262 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java @@ -845,4 +845,9 @@ private Constants() { public static final String SECURITY_CONFIG_TYPE_PASSWORD = "PASSWORD"; public static final String SECURITY_CONFIG_TYPE_LDAP = "LDAP"; + + /** + * use for endKey of RheaKVStore scan + */ + public static final String RANDOM_STRING = "zzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz"; } diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java index 8ad258594f34..5738e1029a8b 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftConnectionStateListener.java @@ -24,6 +24,9 @@ import com.alipay.sofa.jraft.core.Replicator; import com.alipay.sofa.jraft.entity.PeerId; +import lombok.extern.slf4j.Slf4j; + +@Slf4j public class RaftConnectionStateListener implements Replicator.ReplicatorStateListener { private final ConnectionListener connectionListener; private ConnectionState connectionState; @@ -34,17 +37,17 @@ public RaftConnectionStateListener(ConnectionListener connectionListener) { @Override public void onCreated(PeerId peerId) { - + log.info("{}:{} created...", peerId.getIp(), peerId.getPort()); } @Override public void onError(PeerId peerId, Status status) { - + log.error("{}:{} an error occurred, {}", peerId.getIp(), peerId.getPort(), status.getErrorMsg()); } @Override public void onDestroyed(PeerId peerId) { - + log.info("{}:{} destroyed...", peerId.getIp(), peerId.getPort()); } @Override diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java index 161bdd55ce24..c6500971a9f8 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java @@ -73,9 +73,7 @@ public class RaftRegistry implements Registry { private SubscribeListenerManager subscribeListenerManager; - private static final String REGISTRY_DOLPHINSCHEDULER_WORKER_GROUPS = "/nodes/worker-groups"; - - private static final String RANDOM_STRING = RandomStringUtils.randomAlphanumeric(200); + private static final String REGISTRY_DOLPHINSCHEDULER_WORKER_GROUPS = "worker-groups"; private static final String API_TYPE = "api"; @@ -181,16 +179,13 @@ public Collection children(String key) { children = getWorkerGroups(); } else { - final List result = kvStore.bScan(key, key + Constants.SINGLE_SLASH + RANDOM_STRING); + final List result = kvStore.bScan(key, key + Constants.SINGLE_SLASH + Constants.RANDOM_STRING); if (result.isEmpty()) { return new ArrayList<>(); } for (final KVEntry kv : result) { - if (StringUtils.isEmpty(readUtf8(kv.getValue()))) { - continue; - } final String entryKey = readUtf8(kv.getKey()); - if (StringUtils.isEmpty(entryKey)) { + if (StringUtils.isEmpty(readUtf8(kv.getValue())) || StringUtils.isEmpty(entryKey)) { continue; } String child = entryKey.substring(entryKey.lastIndexOf(Constants.SINGLE_SLASH) + 1); diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java index 08cd63ca57ee..9f9946fd718d 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/SubscribeListenerManager.java @@ -33,7 +33,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; import com.alipay.sofa.jraft.rhea.client.RheaKVStore; import com.alipay.sofa.jraft.rhea.storage.KVEntry; @@ -47,8 +46,6 @@ public class SubscribeListenerManager implements AutoCloseable { private final RheaKVStore kvStore; - private static final String REGEX = "\\d{1,3}(\\.\\d{1,3}){3,5}:\\d+"; - public SubscribeListenerManager(RaftRegistryProperties properties, RheaKVStore kvStore) { this.properties = properties; this.kvStore = kvStore; @@ -85,11 +82,8 @@ private class SubscribeCheckTask implements Runnable { @Override public void run() { - if (dataSubScribeMap.isEmpty()) { - return; - } final Map currentNodeDataMap = getNodeDataMap(); - if (currentNodeDataMap.isEmpty()) { + if (dataSubScribeMap.isEmpty() || currentNodeDataMap.isEmpty()) { return; } // find the different @@ -116,6 +110,10 @@ public void run() { nodeDataMap.clear(); nodeDataMap.putAll(currentNodeDataMap); // trigger listener + triggerListener(addedData, deletedData, updatedData); + } + + private void triggerListener(Map addedData, Map deletedData, Map updatedData) { for (Map.Entry> entry : dataSubScribeMap.entrySet()) { String subscribeKey = entry.getKey(); List subscribeListeners = entry.getValue(); @@ -136,18 +134,17 @@ public void run() { * @return */ private Map getNodeDataMap() { - Pattern pattern = Pattern.compile(REGEX); - HashMap nodeDataMap = new HashMap<>(); - final List entryList = kvStore.bScan(Constants.REGISTRY_DOLPHINSCHEDULER_NODE + Constants.SINGLE_SLASH, null); + HashMap dataMap = new HashMap<>(); + final List entryList = kvStore.bScan(Constants.REGISTRY_DOLPHINSCHEDULER_NODE, Constants.REGISTRY_DOLPHINSCHEDULER_NODE + Constants.SINGLE_SLASH + Constants.RANDOM_STRING); for (KVEntry kvEntry : entryList) { final String value = readUtf8(kvEntry.getValue()); final String entryKey = readUtf8(kvEntry.getKey()); - if (StringUtils.isEmpty(value) || pattern.matcher(value).find() || !entryKey.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_NODE)) { + if (StringUtils.isEmpty(value) || !entryKey.startsWith(Constants.REGISTRY_DOLPHINSCHEDULER_NODE)) { continue; } - nodeDataMap.put(entryKey, value); + dataMap.put(entryKey, value); } - return nodeDataMap; + return dataMap; } private void triggerListener(Map nodeDataMap, String subscribeKey, List subscribeListeners, Event.Type type) { From 43e6ebe669663c7c4c3093d59a976b374c5f1bb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=B1=E5=AD=A6=E9=80=9A?= <594754793@qq.com> Date: Mon, 1 Aug 2022 22:35:43 +0800 Subject: [PATCH 9/9] fix --- .../dolphinscheduler/plugin/registry/raft/RaftRegistry.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java index c6500971a9f8..cc4648b6805f 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-raft/src/main/java/org/apache/dolphinscheduler/plugin/registry/raft/RaftRegistry.java @@ -27,8 +27,6 @@ import org.apache.dolphinscheduler.registry.api.SubscribeListener; import org.apache.dolphinscheduler.spi.utils.StringUtils; -import org.apache.commons.lang3.RandomStringUtils; - import java.util.ArrayList; import java.util.Collection; import java.util.Comparator;