tmpWorker =
+ weightedServers.get((robinIndex.incrementAndGet()) % weightedServers.size());
+ tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() + tmpWorker.getWeight());
+
+ if (tmpWorker.getCurrentWeight() >= totalWeight) {
+ tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() - totalWeight);
+ selectedWorker = tmpWorker;
+ }
+ }
+
+ return Optional.of(selectedWorker.getServer().getAddress());
+ }
+
+ @Override
+ public WorkerLoadBalancerType getType() {
+ return WorkerLoadBalancerType.DYNAMIC_WEIGHTED_ROUND_ROBIN;
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancer.java
new file mode 100644
index 0000000000..92840ee7ce
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancer.java
@@ -0,0 +1,109 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.IClusters;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * This load balancer uses a fixed weighted round robin algorithm to select a worker from {@link WorkerClusters}.
+ *
+ * e.g. if there are 3 worker servers with weight 1, 2, 3, the selection will be like:
+ * 1, 2, 3, 1, 2, 3, 3, 1, 2, 3, 3, 1, 2, 3, ...
+ *
+ * The weight of each worker server is determined by the worker server itself.
+ */
+public class FixedWeightedRoundRobinWorkerLoadBalancer implements IWorkerLoadBalancer {
+
+ private final WorkerClusters workerClusters;
+
+ private final AtomicInteger robinIndex = new AtomicInteger(0);
+
+ private final Map> weightedServerMap = new ConcurrentHashMap<>();
+
+ public FixedWeightedRoundRobinWorkerLoadBalancer(WorkerClusters workerClusters) {
+ this.workerClusters = workerClusters;
+ this.workerClusters.registerListener(new IClusters.IClustersChangeListener() {
+
+ @Override
+ public void onServerAdded(WorkerServerMetadata server) {
+ weightedServerMap.put(server.getAddress(), new WeightedServer<>(server, server.getWorkerWeight()));
+ }
+
+ @Override
+ public void onServerRemove(WorkerServerMetadata server) {
+ weightedServerMap.remove(server.getAddress(), new WeightedServer<>(server, server.getWorkerWeight()));
+ }
+
+ @Override
+ public void onServerUpdate(WorkerServerMetadata server) {
+ // don't care the update event, since this will not affect the weight
+ }
+ });
+ }
+
+ @Override
+ public Optional select(@NotNull String workerGroup) {
+ List> weightedServers =
+ workerClusters.getNormalWorkerServerAddressByGroup(workerGroup)
+ .stream()
+ .map(weightedServerMap::get)
+ // filter non null here to avoid the two map changed between
+ // workerClusters and weightedServerMap is not atomic
+ .filter(Objects::nonNull)
+ .collect(Collectors.toList());
+ if (CollectionUtils.isEmpty(weightedServers)) {
+ return Optional.empty();
+ }
+
+ double totalWeight = weightedServers.stream().mapToDouble(WeightedServer::getWeight).sum();
+
+ WeightedServer selectedWorker = null;
+ while (selectedWorker == null) {
+ WeightedServer tmpWorker =
+ weightedServers.get((robinIndex.incrementAndGet()) % weightedServers.size());
+ tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() + tmpWorker.getWeight());
+
+ if (tmpWorker.getCurrentWeight() >= totalWeight) {
+ tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() - totalWeight);
+ selectedWorker = tmpWorker;
+ }
+ }
+
+ return Optional.of(selectedWorker.getServer().getAddress());
+ }
+
+ @Override
+ public WorkerLoadBalancerType getType() {
+ return WorkerLoadBalancerType.FIXED_WEIGHTED_ROUND_ROBIN;
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterInfoChangeListener.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/IWorkerLoadBalancer.java
similarity index 56%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterInfoChangeListener.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/IWorkerLoadBalancer.java
index ffd7628923..d244b9ed2a 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterInfoChangeListener.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/IWorkerLoadBalancer.java
@@ -15,21 +15,27 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.registry;
+package org.apache.dolphinscheduler.server.master.cluster.loadbalancer;
-import org.apache.dolphinscheduler.common.model.MasterHeartBeat;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
-import java.util.Map;
+import java.util.Optional;
+
+import lombok.NonNull;
/**
- * The listener used in {@link ServerNodeManager} to notify the change of master info.
+ * The worker load balancer used to select a worker from the {@link WorkerClusters} by load balancer algorithm.
*/
-public interface MasterInfoChangeListener {
+public interface IWorkerLoadBalancer {
/**
- * Used to notify the change of master info.
+ * Select a worker address under the given worker group.
*
- * @param masterNodeInfo master node info map, key is master address, value is master info.
+ * @param workerGroup worker group cannot be null.
+ * @return the selected worker address, or empty if no worker is available.
*/
- void notify(Map masterNodeInfo);
+ Optional select(@NonNull String workerGroup);
+
+ WorkerLoadBalancerType getType();
+
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancer.java
new file mode 100644
index 0000000000..df0d074b7c
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * The worker load balancer used to select a worker from the {@link WorkerClusters} by random algorithm.
+ */
+public class RandomWorkerLoadBalancer implements IWorkerLoadBalancer {
+
+ private final WorkerClusters workerClusters;
+
+ public RandomWorkerLoadBalancer(WorkerClusters workerClusters) {
+ this.workerClusters = workerClusters;
+ }
+
+ @Override
+ public Optional select(@NotNull String workerGroup) {
+ List workerServerAddresses = workerClusters.getNormalWorkerServerAddressByGroup(workerGroup);
+ if (CollectionUtils.isEmpty(workerServerAddresses)) {
+ return Optional.empty();
+ }
+ int index = ThreadLocalRandom.current().nextInt(workerServerAddresses.size());
+ return Optional.of(workerServerAddresses.get(index));
+ }
+
+ @Override
+ public WorkerLoadBalancerType getType() {
+ return WorkerLoadBalancerType.RANDOM;
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancer.java
new file mode 100644
index 0000000000..767e67a928
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * The worker load balancer used to select a worker from the {@link WorkerClusters} by round-robin algorithm.
+ */
+public class RoundRobinWorkerLoadBalancer implements IWorkerLoadBalancer {
+
+ private final WorkerClusters workerClusters;
+
+ private final AtomicInteger robinIndex = new AtomicInteger(0);
+
+ public RoundRobinWorkerLoadBalancer(WorkerClusters workerClusters) {
+ this.workerClusters = workerClusters;
+ }
+
+ @Override
+ public Optional select(@NotNull String workerGroup) {
+ List workerServerAddresses = workerClusters.getNormalWorkerServerAddressByGroup(workerGroup);
+ if (CollectionUtils.isEmpty(workerServerAddresses)) {
+ return Optional.empty();
+ }
+ return Optional.of(workerServerAddresses.get(robinIndex.getAndIncrement() % workerServerAddresses.size()));
+ }
+
+ @Override
+ public WorkerLoadBalancerType getType() {
+ return WorkerLoadBalancerType.ROUND_ROBIN;
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WeightedServer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WeightedServer.java
new file mode 100644
index 0000000000..04a5e19378
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WeightedServer.java
@@ -0,0 +1,45 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.IClusters;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+@Data
+@Builder
+@AllArgsConstructor
+@NoArgsConstructor
+public class WeightedServer {
+
+ private T server;
+
+ private double weight;
+
+ private double currentWeight;
+
+ public WeightedServer(T server, double weight) {
+ this.server = server;
+ this.weight = weight;
+ this.currentWeight = 0; // Initialize currentWeight is 0
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfiguration.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfiguration.java
new file mode 100644
index 0000000000..9ef16d8287
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfiguration.java
@@ -0,0 +1,50 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.ClusterManager;
+import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+
+@Configuration
+public class WorkerLoadBalancerConfiguration {
+
+ @Bean
+ public IWorkerLoadBalancer randomWorkerLoadBalancer(MasterConfig masterConfig, ClusterManager clusterManager) {
+ WorkerLoadBalancerConfigurationProperties workerLoadBalancerConfigurationProperties =
+ masterConfig.getWorkerLoadBalancerConfigurationProperties();
+ switch (workerLoadBalancerConfigurationProperties.getType()) {
+ case RANDOM:
+ return new RandomWorkerLoadBalancer(clusterManager.getWorkerClusters());
+ case ROUND_ROBIN:
+ return new RoundRobinWorkerLoadBalancer(clusterManager.getWorkerClusters());
+ case FIXED_WEIGHTED_ROUND_ROBIN:
+ return new FixedWeightedRoundRobinWorkerLoadBalancer(clusterManager.getWorkerClusters());
+ case DYNAMIC_WEIGHTED_ROUND_ROBIN:
+ return new DynamicWeightedRoundRobinWorkerLoadBalancer(
+ clusterManager.getWorkerClusters(),
+ workerLoadBalancerConfigurationProperties.getDynamicWeightConfigProperties());
+ default:
+ throw new IllegalArgumentException(
+ "unSupport worker load balancer type " + workerLoadBalancerConfigurationProperties.getType());
+ }
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfigurationProperties.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfigurationProperties.java
new file mode 100644
index 0000000000..bcc905617f
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerConfigurationProperties.java
@@ -0,0 +1,61 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import lombok.Data;
+
+import org.springframework.validation.Errors;
+
+@Data
+public class WorkerLoadBalancerConfigurationProperties {
+
+ private WorkerLoadBalancerType type = WorkerLoadBalancerType.ROUND_ROBIN;
+
+ private DynamicWeightConfigProperties dynamicWeightConfigProperties = new DynamicWeightConfigProperties();
+
+ public void validate(Errors errors) {
+ dynamicWeightConfigProperties.validated(errors);
+ }
+
+ @Data
+ public static class DynamicWeightConfigProperties {
+
+ private int cpuUsageWeight = 30;
+
+ private int memoryUsageWeight = 30;
+
+ private int taskThreadPoolUsageWeight = 40;
+
+ public void validated(Errors errors) {
+ if (cpuUsageWeight < 0) {
+ errors.rejectValue("cpuUsageWeight", "cpuUsageWeight", "cpuUsageWeight must >= 0");
+ }
+ if (memoryUsageWeight < 0) {
+ errors.rejectValue("memoryUsageWeight", "memoryUsageWeight", "memoryUsageWeight must >= 0");
+ }
+ if (taskThreadPoolUsageWeight < 0) {
+ errors.rejectValue("threadUsageWeight", "threadUsageWeight", "threadUsageWeight must >= 0");
+ }
+ if (cpuUsageWeight + memoryUsageWeight + taskThreadPoolUsageWeight != 100) {
+ errors.rejectValue("cpuUsageWeight", "cpuUsageWeight",
+ "cpuUsageWeight + memoryUsageWeight + threadUsageWeight must be 100");
+ }
+ }
+
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostSelector.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerType.java
similarity index 82%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostSelector.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerType.java
index 7f28c2bc33..64bdfdef7c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostSelector.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/WorkerLoadBalancerType.java
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
+package org.apache.dolphinscheduler.server.master.cluster.loadbalancer;
-public enum HostSelector {
+public enum WorkerLoadBalancerType {
RANDOM,
-
ROUND_ROBIN,
-
- LOWER_WEIGHT
+ FIXED_WEIGHTED_ROUND_ROBIN,
+ DYNAMIC_WEIGHTED_ROUND_ROBIN,
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/CommandFetcherConfiguration.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/CommandFetcherConfiguration.java
index 4a4d3c1efc..c049ec8a02 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/CommandFetcherConfiguration.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/CommandFetcherConfiguration.java
@@ -20,9 +20,9 @@ package org.apache.dolphinscheduler.server.master.command;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.dao.repository.CommandDao;
+import org.apache.dolphinscheduler.server.master.cluster.MasterSlotManager;
import org.apache.dolphinscheduler.server.master.config.CommandFetchStrategy;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.registry.MasterSlotManager;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/IdSlotBasedCommandFetcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/IdSlotBasedCommandFetcher.java
index a417820093..fa1ad270cb 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/IdSlotBasedCommandFetcher.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/command/IdSlotBasedCommandFetcher.java
@@ -19,9 +19,9 @@ package org.apache.dolphinscheduler.server.master.command;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.repository.CommandDao;
+import org.apache.dolphinscheduler.server.master.cluster.MasterSlotManager;
import org.apache.dolphinscheduler.server.master.config.CommandFetchStrategy;
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
-import org.apache.dolphinscheduler.server.master.registry.MasterSlotManager;
import java.util.Collections;
import java.util.List;
@@ -51,13 +51,13 @@ public class IdSlotBasedCommandFetcher implements ICommandFetcher {
@Override
public List fetchCommands() {
long scheduleStartTime = System.currentTimeMillis();
- int currentSlotIndex = masterSlotManager.getSlot();
- int totalSlot = masterSlotManager.getMasterSize();
- if (totalSlot <= 0 || currentSlotIndex < 0) {
- log.warn("Slot is validated, current master slots: {}, the current slot index is {}", totalSlot,
- currentSlotIndex);
+ if (!masterSlotManager.checkSlotValid()) {
+ log.warn("MasterSlotManager check slot ({} -> {})is invalidated.",
+ masterSlotManager.getCurrentMasterSlot(), masterSlotManager.getTotalMasterSlots());
return Collections.emptyList();
}
+ int currentSlotIndex = masterSlotManager.getCurrentMasterSlot();
+ int totalSlot = masterSlotManager.getTotalMasterSlots();
List commands = commandDao.queryCommandByIdSlot(
currentSlotIndex,
totalSlot,
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
index d9e6ab3b7d..8e72b1769e 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
@@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.server.master.config;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.registry.api.ConnectStrategyProperties;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostSelector;
+import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.WorkerLoadBalancerConfigurationProperties;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
@@ -63,14 +63,6 @@ public class MasterConfig implements Validator {
private int masterSyncTaskExecutorThreadPoolSize = Runtime.getRuntime().availableProcessors();
private int masterAsyncTaskExecutorThreadPoolSize = Runtime.getRuntime().availableProcessors();
- /**
- * The task dispatch thread pool size.
- */
- private int dispatchTaskNumber = 3;
- /**
- * Worker select strategy.
- */
- private HostSelector hostSelector = HostSelector.LOWER_WEIGHT;
/**
* Master heart beat task execute interval.
*/
@@ -96,6 +88,9 @@ public class MasterConfig implements Validator {
private CommandFetchStrategy commandFetchStrategy = new CommandFetchStrategy();
+ private WorkerLoadBalancerConfigurationProperties workerLoadBalancerConfigurationProperties =
+ new WorkerLoadBalancerConfigurationProperties();
+
/**
* The IP address and listening port of the master server in the format 'ip:listenPort'.
*/
@@ -123,9 +118,6 @@ public class MasterConfig implements Validator {
if (masterConfig.getExecThreads() <= 0) {
errors.rejectValue("exec-threads", null, "should be a positive value");
}
- if (masterConfig.getDispatchTaskNumber() <= 0) {
- errors.rejectValue("dispatch-task-number", null, "should be a positive value");
- }
if (masterConfig.getMaxHeartbeatInterval().toMillis() < 0) {
errors.rejectValue("max-heartbeat-interval", null, "should be a valid duration");
}
@@ -149,6 +141,7 @@ public class MasterConfig implements Validator {
masterConfig.setMasterAddress(NetUtils.getAddr(masterConfig.getListenPort()));
}
commandFetchStrategy.validate(errors);
+ workerLoadBalancerConfigurationProperties.validate(errors);
masterConfig.setMasterRegistryPath(
RegistryNodeType.MASTER.getRegistryPath() + "/" + masterConfig.getMasterAddress());
@@ -161,8 +154,6 @@ public class MasterConfig implements Validator {
"\n listen-port -> " + listenPort +
"\n pre-exec-threads -> " + preExecThreads +
"\n exec-threads -> " + execThreads +
- "\n dispatch-task-number -> " + dispatchTaskNumber +
- "\n host-selector -> " + hostSelector +
"\n max-heartbeat-interval -> " + maxHeartbeatInterval +
"\n task-commit-retry-times -> " + taskCommitRetryTimes +
"\n task-commit-interval -> " + taskCommitInterval +
@@ -175,6 +166,8 @@ public class MasterConfig implements Validator {
"\n master-registry-path: " + masterRegistryPath +
"\n worker-group-refresh-interval: " + workerGroupRefreshInterval +
"\n command-fetch-strategy: " + commandFetchStrategy +
+ "\n worker-load-balancer-configuration-properties: "
+ + workerLoadBalancerConfigurationProperties +
"\n****************************Master Configuration**************************************";
log.info(config);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/ExecuteException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/ExecuteException.java
deleted file mode 100644
index abc70e4035..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/ExecuteException.java
+++ /dev/null
@@ -1,47 +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.server.master.dispatch.exceptions;
-
-/**
- * execute exception
- */
-public class ExecuteException extends Exception {
-
- public ExecuteException() {
- super();
- }
-
- public ExecuteException(String message) {
- super(message);
- }
-
- public ExecuteException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public ExecuteException(Throwable cause) {
- super(cause);
- }
-
- protected ExecuteException(String message, Throwable cause,
- boolean enableSuppression,
- boolean writableStackTrace) {
- super(message, cause, enableSuppression, writableStackTrace);
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java
deleted file mode 100644
index 5ca63f12e3..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java
+++ /dev/null
@@ -1,69 +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.server.master.dispatch.host;
-
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
-import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
-
-import org.apache.commons.collections4.CollectionUtils;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Optional;
-import java.util.Set;
-
-import org.springframework.beans.factory.annotation.Autowired;
-
-/**
- * common host manager
- */
-public abstract class CommonHostManager implements HostManager {
-
- /**
- * server node manager
- */
- @Autowired
- protected ServerNodeManager serverNodeManager;
-
- @Override
- public Optional select(String workerGroup) throws WorkerGroupNotFoundException {
- List candidates = getWorkerCandidates(workerGroup);
- if (CollectionUtils.isEmpty(candidates)) {
- return Optional.empty();
- }
- return Optional.ofNullable(select(candidates));
- }
-
- protected abstract HostWorker select(Collection nodes);
-
- protected List getWorkerCandidates(String workerGroup) throws WorkerGroupNotFoundException {
- List hostWorkers = new ArrayList<>();
- Set nodes = serverNodeManager.getWorkerGroupNodes(workerGroup);
- if (CollectionUtils.isNotEmpty(nodes)) {
- for (String node : nodes) {
- serverNodeManager.getWorkerNodeInfo(node).ifPresent(
- workerNodeInfo -> hostWorkers
- .add(HostWorker.of(node, workerNodeInfo.getWorkerHostWeight(), workerGroup)));
- }
- }
- return hostWorkers;
- }
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManagerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManagerConfig.java
deleted file mode 100644
index 10d9e0f491..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManagerConfig.java
+++ /dev/null
@@ -1,64 +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.server.master.dispatch.host;
-
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostSelector;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.beans.factory.config.AutowireCapableBeanFactory;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.Configuration;
-
-/**
- * host manager config
- */
-@Configuration
-public class HostManagerConfig {
-
- private AutowireCapableBeanFactory beanFactory;
-
- @Autowired
- private MasterConfig masterConfig;
-
- @Autowired
- public HostManagerConfig(AutowireCapableBeanFactory beanFactory) {
- this.beanFactory = beanFactory;
- }
-
- @Bean
- public HostManager hostManager() {
- HostSelector selector = masterConfig.getHostSelector();
- HostManager hostManager;
- switch (selector) {
- case RANDOM:
- hostManager = new RandomHostManager();
- break;
- case ROUND_ROBIN:
- hostManager = new RoundRobinHostManager();
- break;
- case LOWER_WEIGHT:
- hostManager = new LowerWeightHostManager();
- break;
- default:
- throw new IllegalArgumentException("unSupport selector " + selector);
- }
- beanFactory.autowireBean(hostManager);
- return hostManager;
- }
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java
deleted file mode 100644
index e14cd26a9f..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java
+++ /dev/null
@@ -1,165 +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.server.master.dispatch.host;
-
-import org.apache.dolphinscheduler.common.enums.ServerStatus;
-import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWeight;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.LowerWeightRoundRobin;
-import org.apache.dolphinscheduler.server.master.registry.WorkerInfoChangeListener;
-
-import org.apache.commons.collections4.CollectionUtils;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import javax.annotation.PostConstruct;
-
-import lombok.extern.slf4j.Slf4j;
-
-/**
- * lower weight host manager
- */
-@Slf4j
-public class LowerWeightHostManager extends CommonHostManager {
-
- /**
- * selector
- */
- private LowerWeightRoundRobin selector;
-
- /**
- * worker host weights
- */
- private ConcurrentHashMap> workerHostWeightsMap;
-
- private final ReentrantReadWriteLock workerGroupLock = new ReentrantReadWriteLock();
-
- private final ReentrantReadWriteLock.ReadLock workerGroupReadLock = workerGroupLock.readLock();
- private final ReentrantReadWriteLock.WriteLock workerGroupWriteLock = workerGroupLock.writeLock();
-
- @PostConstruct
- public void init() {
- this.selector = new LowerWeightRoundRobin();
- this.workerHostWeightsMap = new ConcurrentHashMap<>();
- serverNodeManager.addWorkerInfoChangeListener(new WorkerWeightListener());
- }
-
- @Override
- public Optional select(String workerGroup) throws WorkerGroupNotFoundException {
- Set workerHostWeights = getWorkerHostWeights(workerGroup);
- if (CollectionUtils.isNotEmpty(workerHostWeights)) {
- return Optional.ofNullable(selector.select(workerHostWeights).getHost());
- }
- return Optional.empty();
- }
-
- @Override
- public HostWorker select(Collection nodes) {
- throw new UnsupportedOperationException("not support");
- }
-
- private class WorkerWeightListener implements WorkerInfoChangeListener {
-
- @Override
- public void notify(Map> workerGroups, Map workerNodeInfo) {
- syncWorkerResources(workerGroups, workerNodeInfo);
- }
-
- /**
- * Sync worker resource.
- *
- * @param workerGroupNodes worker group nodes, key is worker group, value is worker group nodes.
- * @param workerNodeInfoMap worker node info map, key is worker node, value is worker info.
- */
- private void syncWorkerResources(final Map> workerGroupNodes,
- final Map workerNodeInfoMap) {
- try {
- Map> workerHostWeights = new HashMap<>();
- for (Map.Entry> entry : workerGroupNodes.entrySet()) {
- String workerGroup = entry.getKey();
- Set nodes = entry.getValue();
- Set hostWeights = new HashSet<>(nodes.size());
- for (String node : nodes) {
- WorkerHeartBeat heartbeat = workerNodeInfoMap.getOrDefault(node, null);
- Optional hostWeightOpt = getHostWeight(node, workerGroup, heartbeat);
- hostWeightOpt.ifPresent(hostWeights::add);
- }
- if (!hostWeights.isEmpty()) {
- workerHostWeights.put(workerGroup, hostWeights);
- }
- }
- syncWorkerHostWeight(workerHostWeights);
- } catch (Throwable ex) {
- log.error("Sync worker resource error", ex);
- }
- }
-
- private void syncWorkerHostWeight(Map> workerHostWeights) {
- workerGroupWriteLock.lock();
- try {
- workerHostWeightsMap.clear();
- workerHostWeightsMap.putAll(workerHostWeights);
- } finally {
- workerGroupWriteLock.unlock();
- }
- }
- }
-
- public Optional getHostWeight(String workerAddress, String workerGroup, WorkerHeartBeat heartBeat) {
- if (heartBeat == null) {
- log.warn("Worker {} in WorkerGroup {} have not received the heartbeat", workerAddress, workerGroup);
- return Optional.empty();
- }
- if (ServerStatus.BUSY == heartBeat.getServerStatus()) {
- log.warn("Worker {} in workerGroup {} is Busy, heartbeat is {}", workerAddress, workerGroup, heartBeat);
- return Optional.empty();
- }
- return Optional.of(
- new HostWeight(
- HostWorker.of(workerAddress, heartBeat.getWorkerHostWeight(), workerGroup),
- heartBeat.getCpuUsage(),
- heartBeat.getMemoryUsage(),
- heartBeat.getDiskUsage(),
- heartBeat.getThreadPoolUsage(),
- heartBeat.getStartupTime()));
- }
-
- private Set getWorkerHostWeights(String workerGroup) throws WorkerGroupNotFoundException {
- workerGroupReadLock.lock();
- try {
- Set hostWeights = workerHostWeightsMap.get(workerGroup);
- if (hostWeights == null) {
- throw new WorkerGroupNotFoundException("Can not find worker group " + workerGroup);
- }
- return hostWeights;
- } finally {
- workerGroupReadLock.unlock();
- }
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RandomHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RandomHostManager.java
deleted file mode 100644
index b4288dc666..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RandomHostManager.java
+++ /dev/null
@@ -1,47 +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.server.master.dispatch.host;
-
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.RandomSelector;
-
-import java.util.Collection;
-
-/**
- * random host manager
- */
-public class RandomHostManager extends CommonHostManager {
-
- /**
- * selector
- */
- private final RandomSelector selector;
-
- /**
- * set round robin
- */
- public RandomHostManager() {
- this.selector = new RandomSelector();
- }
-
- @Override
- public HostWorker select(Collection nodes) {
- return selector.select(nodes);
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManager.java
deleted file mode 100644
index 28c6f0d17d..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManager.java
+++ /dev/null
@@ -1,47 +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.server.master.dispatch.host;
-
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
-import org.apache.dolphinscheduler.server.master.dispatch.host.assign.RoundRobinSelector;
-
-import java.util.Collection;
-
-/**
- * round robin host manager
- */
-public class RoundRobinHostManager extends CommonHostManager {
-
- /**
- * selector
- */
- private final RoundRobinSelector selector;
-
- /**
- * set round robin
- */
- public RoundRobinHostManager() {
- this.selector = new RoundRobinSelector();
- }
-
- @Override
- public HostWorker select(Collection nodes) {
- return selector.select(nodes);
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/AbstractSelector.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/AbstractSelector.java
deleted file mode 100644
index 92169f48b0..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/AbstractSelector.java
+++ /dev/null
@@ -1,47 +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.server.master.dispatch.host.assign;
-
-import org.apache.commons.collections4.CollectionUtils;
-
-import java.util.Collection;
-
-/**
- * AbstractSelector
- */
-public abstract class AbstractSelector implements Selector {
-
- @Override
- public T select(Collection source) {
-
- if (CollectionUtils.isEmpty(source)) {
- throw new IllegalArgumentException("Empty source.");
- }
-
- /**
- * if only one , return directly
- */
- if (source.size() == 1) {
- return (T) source.toArray()[0];
- }
- return doSelect(source);
- }
-
- protected abstract T doSelect(Collection source);
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWeight.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWeight.java
deleted file mode 100644
index 147761f7df..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWeight.java
+++ /dev/null
@@ -1,69 +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.server.master.dispatch.host.assign;
-
-import org.apache.dolphinscheduler.extract.base.utils.Constants;
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-
-import lombok.Data;
-
-@Data
-public class HostWeight {
-
- private final int THREAD_USAGE_FACTOR = 10;
-
- private final int CPU_USAGE_FACTOR = 20;
-
- private final int MEMORY_USAGE_FACTOR = 20;
-
- private final int DISK_USAGE_FACTOR = 50;
-
- private final Host host;
-
- private final double weight;
-
- // if the weight is small, then is will be chosen first
- private double currentWeight;
-
- public HostWeight(HostWorker hostWorker,
- double cpuUsage,
- double memoryUsage,
- double diskUsage,
- double threadPoolUsage,
- long startTime) {
- this.host = hostWorker;
- this.weight = calculateWeight(cpuUsage, memoryUsage, diskUsage, threadPoolUsage, startTime);
- this.currentWeight = this.weight;
- }
-
- private double calculateWeight(double cpuUsage,
- double memoryUsage,
- double diskUsage,
- double threadPoolUsage,
- long startTime) {
- double calculatedWeight = 100 - (cpuUsage * CPU_USAGE_FACTOR + memoryUsage * MEMORY_USAGE_FACTOR
- + diskUsage * DISK_USAGE_FACTOR + threadPoolUsage * THREAD_USAGE_FACTOR);
- long uptime = System.currentTimeMillis() - startTime;
- if (uptime > 0 && uptime < Constants.WARM_UP_TIME) {
- // If the warm-up is not over, add the weight
- return calculatedWeight * Constants.WARM_UP_TIME / uptime;
- }
- return calculatedWeight;
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorker.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorker.java
deleted file mode 100644
index f25b621840..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorker.java
+++ /dev/null
@@ -1,77 +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.server.master.dispatch.host.assign;
-
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-
-/**
- * host worker
- */
-public class HostWorker extends Host {
-
- /**
- * host weight
- */
- private int hostWeight;
-
- /**
- * worker group
- */
- private String workerGroup;
-
- public HostWorker(String ip, int port, int hostWeight, String workerGroup) {
- super(ip, port);
- this.hostWeight = hostWeight;
- this.workerGroup = workerGroup;
- }
-
- public HostWorker(String address, int hostWeight, String workerGroup) {
- super(address);
- this.hostWeight = hostWeight;
- this.workerGroup = workerGroup;
- }
-
- public int getHostWeight() {
- return hostWeight;
- }
-
- public void setHostWeight(int hostWeight) {
- this.hostWeight = hostWeight;
- }
-
- public String getWorkerGroup() {
- return workerGroup;
- }
-
- public void setWorkerGroup(String workerGroup) {
- this.workerGroup = workerGroup;
- }
-
- public static HostWorker of(String address, int hostWeight, String workerGroup) {
- return new HostWorker(address, hostWeight, workerGroup);
- }
-
- @Override
- public String toString() {
- return "Host{"
- + "hostWeight=" + hostWeight
- + ", workerGroup='" + workerGroup + '\''
- + '}';
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java
deleted file mode 100644
index 9c3c4fd696..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java
+++ /dev/null
@@ -1,54 +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.server.master.dispatch.host.assign;
-
-import java.util.Collection;
-
-/**
- * lower weight round robin
- */
-public class LowerWeightRoundRobin extends AbstractSelector {
-
- /**
- * Selects a HostWeight from a collection of HostWeight objects.
- * The selection is based on the current weight of each HostWeight.
- * The HostWeight with the smallest current weight is selected.
- *
- * @param sources A collection of HostWeight objects to select from.
- * @return The selected HostWeight with the smallest current weight.
- */
- @Override
- public HostWeight doSelect(Collection sources) {
- double totalWeight = 0;
- double lowWeight = 0;
- HostWeight lowerNode = null;
- for (HostWeight hostWeight : sources) {
- totalWeight += hostWeight.getWeight();
- hostWeight.setCurrentWeight(hostWeight.getCurrentWeight() + hostWeight.getWeight());
- if (lowerNode == null || lowWeight > hostWeight.getCurrentWeight()) {
- lowerNode = hostWeight;
- lowWeight = hostWeight.getCurrentWeight();
- }
- }
- if (lowerNode != null) {
- lowerNode.setCurrentWeight(lowerNode.getCurrentWeight() + totalWeight);
- }
- return lowerNode;
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelector.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelector.java
deleted file mode 100644
index a0f83232a2..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelector.java
+++ /dev/null
@@ -1,66 +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.server.master.dispatch.host.assign;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * random selector
- */
-public class RandomSelector extends AbstractSelector {
-
- /**
- * This method selects a HostWorker from a collection of HostWorker objects using a weighted random algorithm.
- * The selection is based on the weight of each HostWorker.
- * A random number is generated and the HostWorker whose weight spans this random number is selected.
- *
- * @param source A collection of HostWorker objects to select from.
- * @return The selected HostWorker based on the weighted random algorithm.
- */
- @Override
- public HostWorker doSelect(final Collection source) {
-
- List hosts = new ArrayList<>(source);
- int size = hosts.size();
- int[] weights = new int[size];
- int totalWeight = 0;
- int index = 0;
-
- for (HostWorker host : hosts) {
- totalWeight += host.getHostWeight();
- weights[index] = host.getHostWeight();
- index++;
- }
-
- if (totalWeight > 0) {
- int offset = ThreadLocalRandom.current().nextInt(totalWeight);
-
- for (int i = 0; i < size; i++) {
- offset -= weights[i];
- if (offset < 0) {
- return hosts.get(i);
- }
- }
- }
- return hosts.get(ThreadLocalRandom.current().nextInt(size));
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelector.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelector.java
deleted file mode 100644
index b47eff87b9..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelector.java
+++ /dev/null
@@ -1,148 +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.server.master.dispatch.host.assign;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.springframework.stereotype.Service;
-
-/**
- * Smooth Weight Round Robin
- */
-@Service
-public class RoundRobinSelector extends AbstractSelector {
-
- private ConcurrentMap> workGroupWeightMap =
- new ConcurrentHashMap<>();
-
- private static final int RECYCLE_PERIOD = 100000;
-
- private AtomicBoolean updateLock = new AtomicBoolean();
-
- protected static class WeightedRoundRobin {
-
- private int weight;
- private AtomicLong current = new AtomicLong(0);
- private long lastUpdate;
-
- int getWeight() {
- return weight;
- }
-
- void setWeight(int weight) {
- this.weight = weight;
- current.set(0);
- }
-
- long increaseCurrent() {
- return current.addAndGet(weight);
- }
-
- void sel(int total) {
- current.addAndGet(-1L * total);
- }
-
- long getLastUpdate() {
- return lastUpdate;
- }
-
- void setLastUpdate(long lastUpdate) {
- this.lastUpdate = lastUpdate;
- }
-
- }
-
- /**
- * This method selects a HostWorker from a collection of HostWorker objects using a weighted round-robin algorithm.
- * The selection is based on the current weight of each HostWorker.
- * The HostWorker with the highest current weight is selected.
- *
- * @param source A collection of HostWorker objects to select from.
- * @return The selected HostWorker with the highest current weight.
- */
- @Override
- public HostWorker doSelect(Collection source) {
-
- List hosts = new ArrayList<>(source);
- String key = hosts.get(0).getWorkerGroup();
- ConcurrentMap map = workGroupWeightMap.get(key);
- if (map == null) {
- workGroupWeightMap.putIfAbsent(key, new ConcurrentHashMap<>());
- map = workGroupWeightMap.get(key);
- }
-
- int totalWeight = 0;
- long maxCurrent = Long.MIN_VALUE;
- long now = System.currentTimeMillis();
- HostWorker selectedHost = null;
- WeightedRoundRobin selectWeightRoundRobin = null;
-
- for (HostWorker host : hosts) {
- String workGroupHost = host.getWorkerGroup() + host.getAddress();
- WeightedRoundRobin weightedRoundRobin = map.get(workGroupHost);
- int weight = host.getHostWeight();
- if (weight < 0) {
- weight = 0;
- }
-
- if (weightedRoundRobin == null) {
- weightedRoundRobin = new WeightedRoundRobin();
- // set weight
- weightedRoundRobin.setWeight(weight);
- map.putIfAbsent(workGroupHost, weightedRoundRobin);
- weightedRoundRobin = map.get(workGroupHost);
- }
- if (weight != weightedRoundRobin.getWeight()) {
- weightedRoundRobin.setWeight(weight);
- }
-
- long cur = weightedRoundRobin.increaseCurrent();
- weightedRoundRobin.setLastUpdate(now);
- if (cur > maxCurrent) {
- maxCurrent = cur;
- selectedHost = host;
- selectWeightRoundRobin = weightedRoundRobin;
- }
-
- totalWeight += weight;
- }
-
- if (!updateLock.get() && hosts.size() != map.size() && updateLock.compareAndSet(false, true)) {
- try {
- ConcurrentMap newMap = new ConcurrentHashMap<>(map);
- newMap.entrySet().removeIf(item -> now - item.getValue().getLastUpdate() > RECYCLE_PERIOD);
- workGroupWeightMap.put(key, newMap);
- } finally {
- updateLock.set(false);
- }
- }
-
- if (selectedHost != null) {
- selectWeightRoundRobin.sel(totalWeight);
- return selectedHost;
- }
-
- return hosts.get(0);
- }
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/TaskDispatchException.java
similarity index 87%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/TaskDispatchException.java
index 259822e80b..4c438d72e0 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/TaskDispatchException.java
@@ -15,7 +15,9 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.exception;
+package org.apache.dolphinscheduler.server.master.exception.dispatch;
+
+import org.apache.dolphinscheduler.server.master.exception.MasterException;
public class TaskDispatchException extends MasterException {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/WorkerGroupNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java
similarity index 86%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/WorkerGroupNotFoundException.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java
index 81755152c3..c9d1301725 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/exceptions/WorkerGroupNotFoundException.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.dispatch.exceptions;
+package org.apache.dolphinscheduler.server.master.exception.dispatch;
-public class WorkerGroupNotFoundException extends ExecuteException {
+public class WorkerGroupNotFoundException extends TaskDispatchException {
public WorkerGroupNotFoundException(String workerGroup) {
super("Cannot find worker group: " + workerGroup);
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManager.java
deleted file mode 100644
index 155b973117..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManager.java
+++ /dev/null
@@ -1,114 +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.server.master.registry;
-
-import org.apache.dolphinscheduler.common.enums.ServerStatus;
-import org.apache.dolphinscheduler.common.model.MasterHeartBeat;
-import org.apache.dolphinscheduler.common.model.Server;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.service.queue.MasterPriorityQueue;
-
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.stream.Collectors;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Service;
-
-@Service
-@Slf4j
-public class MasterSlotManager {
-
- @Autowired
- protected ServerNodeManager serverNodeManager;
-
- @Autowired
- protected MasterConfig masterConfig;
-
- private volatile int currentSlot = 0;
- private volatile int totalSlot = 0;
-
- public void start() {
- serverNodeManager.addMasterInfoChangeListener(new MasterSlotManager.SlotChangeListener());
- }
-
- public int getSlot() {
- return currentSlot;
- }
-
- public int getMasterSize() {
- return totalSlot;
- }
-
- public class SlotChangeListener implements MasterInfoChangeListener {
-
- private final Lock slotLock = new ReentrantLock();
-
- private final MasterPriorityQueue masterPriorityQueue = new MasterPriorityQueue();
-
- @Override
- public void notify(Map masterNodeInfo) {
- List serverList = masterNodeInfo.values().stream()
- .filter(heartBeat -> !heartBeat.getServerStatus().equals(ServerStatus.BUSY))
- .map(this::convertHeartBeatToServer)
- .collect(Collectors.toList());
- syncMasterNodes(serverList);
- }
-
- /**
- * sync master nodes
- */
- private void syncMasterNodes(List masterNodes) {
- slotLock.lock();
- try {
- this.masterPriorityQueue.clear();
- this.masterPriorityQueue.putAll(masterNodes);
- int tempCurrentSlot = masterPriorityQueue.getIndex(masterConfig.getMasterAddress());
- int tempTotalSlot = masterNodes.size();
- if (tempCurrentSlot < 0) {
- totalSlot = 0;
- currentSlot = 0;
- log.warn("Current master is not in active master list");
- } else if (tempCurrentSlot != currentSlot || tempTotalSlot != totalSlot) {
- totalSlot = tempTotalSlot;
- currentSlot = tempCurrentSlot;
- log.info("Update master nodes, total master size: {}, current slot: {}", totalSlot, currentSlot);
- }
- } finally {
- slotLock.unlock();
- }
- }
-
- private Server convertHeartBeatToServer(MasterHeartBeat masterHeartBeat) {
- Server server = new Server();
- server.setCreateTime(new Date(masterHeartBeat.getStartupTime()));
- server.setLastHeartbeatTime(new Date(masterHeartBeat.getReportTime()));
- server.setId(masterHeartBeat.getProcessId());
- server.setHost(masterHeartBeat.getHost());
- server.setPort(masterHeartBeat.getPort());
-
- return server;
- }
-
- }
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java
deleted file mode 100644
index d532168b2a..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java
+++ /dev/null
@@ -1,374 +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.server.master.registry;
-
-import org.apache.dolphinscheduler.common.constants.Constants;
-import org.apache.dolphinscheduler.common.model.MasterHeartBeat;
-import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
-import org.apache.dolphinscheduler.common.thread.ThreadUtils;
-import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
-import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper;
-import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
-import org.apache.dolphinscheduler.registry.api.Event;
-import org.apache.dolphinscheduler.registry.api.Event.Type;
-import org.apache.dolphinscheduler.registry.api.RegistryClient;
-import org.apache.dolphinscheduler.registry.api.SubscribeListener;
-import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager;
-
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.ArrayUtils;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import javax.annotation.PreDestroy;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.InitializingBean;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Service;
-
-@Service
-@Slf4j
-public class ServerNodeManager implements InitializingBean {
-
- private final ReentrantReadWriteLock workerGroupLock = new ReentrantReadWriteLock();
- private final ReentrantReadWriteLock.ReadLock workerGroupReadLock = workerGroupLock.readLock();
- private final ReentrantReadWriteLock.WriteLock workerGroupWriteLock = workerGroupLock.writeLock();
-
- private final ReentrantReadWriteLock workerNodeInfoLock = new ReentrantReadWriteLock();
- private final ReentrantReadWriteLock.ReadLock workerNodeInfoReadLock = workerNodeInfoLock.readLock();
- private final ReentrantReadWriteLock.WriteLock workerNodeInfoWriteLock = workerNodeInfoLock.writeLock();
-
- private final ReentrantLock masterNodeInfoLock = new ReentrantLock();
-
- /**
- * worker group nodes, workerGroup -> ips, combining registryWorkerGroupNodes and dbWorkerGroupNodes
- */
- private final ConcurrentHashMap> workerGroupNodes = new ConcurrentHashMap<>();
-
- private final Map masterNodeInfo = new HashMap<>();
-
- private final Map workerNodeInfo = new HashMap<>();
-
- /**
- * executor service
- */
- private ScheduledExecutorService executorService;
-
- @Autowired
- private RegistryClient registryClient;
-
- @Autowired
- private WorkerGroupMapper workerGroupMapper;
-
- @Autowired
- private AlertDao alertDao;
-
- @Autowired
- private MasterConfig masterConfig;
-
- @Autowired
- private ListenerEventAlertManager listenerEventAlertManager;
-
- private final List workerInfoChangeListeners = new ArrayList<>();
-
- private final List masterInfoChangeListeners = new ArrayList<>();
-
- @Override
- public void afterPropertiesSet() {
-
- // load nodes from zookeeper
- refreshNodesAndGroupMappings();
-
- // init executor service
- executorService = Executors
- .newSingleThreadScheduledExecutor(ThreadUtils.newDaemonThreadFactory("ServerNodeManagerExecutor"));
- executorService.scheduleWithFixedDelay(
- new WorkerNodeInfoAndGroupDbSyncTask(),
- 0,
- masterConfig.getWorkerGroupRefreshInterval().getSeconds(),
- TimeUnit.SECONDS);
-
- registryClient.subscribe(RegistryNodeType.MASTER.getRegistryPath(), new MasterDataListener());
- registryClient.subscribe(RegistryNodeType.WORKER.getRegistryPath(), new WorkerDataListener());
- }
-
- class WorkerNodeInfoAndGroupDbSyncTask implements Runnable {
-
- @Override
- public void run() {
- try {
- // sync worker node info
- refreshNodesAndGroupMappings();
- } catch (Exception e) {
- log.error("WorkerNodeInfoAndGroupDbSyncTask error:", e);
- }
- }
- }
-
- /**
- * Refresh master/worker nodes and worker group mapping information
- */
- private void refreshNodesAndGroupMappings() {
- updateWorkerNodes();
- updateWorkerGroupMappings();
- notifyWorkerInfoChangeListeners();
-
- updateMasterNodes();
- notifyMasterInfoChangeListeners();
- }
-
- /**
- * worker group node listener
- */
- class WorkerDataListener implements SubscribeListener {
-
- @Override
- public void notify(Event event) {
- final String path = event.path();
- final Type type = event.type();
- final String data = event.data();
- if (registryClient.isWorkerPath(path)) {
- try {
- String[] parts = path.split("/");
- final String workerAddress = parts[parts.length - 1];
-
- // todo: update workerNodeInfo
- log.debug("received subscribe event : {}", event);
- if (type == Type.ADD) {
- log.info("Worker: {} added, currentNode : {}", path, workerAddress);
- } else if (type == Type.REMOVE) {
- log.info("Worker node : {} down.", path);
- removeSingleWorkerNode(workerAddress);
- alertDao.sendServerStoppedAlert(1, path, "WORKER");
- listenerEventAlertManager.publishServerDownListenerEvent(path, "WORKER");
- } else if (type == Type.UPDATE) {
- syncSingleWorkerNodeInfo(workerAddress, JSONUtils.parseObject(data, WorkerHeartBeat.class));
- }
- } catch (Exception ex) {
- log.error("WorkerGroupListener capture data change and get data failed", ex);
- }
- }
- }
-
- private void syncSingleWorkerNodeInfo(String workerAddress, WorkerHeartBeat info) {
- workerNodeInfoWriteLock.lock();
- try {
- workerNodeInfo.put(workerAddress, info);
- } finally {
- workerNodeInfoWriteLock.unlock();
- }
- }
-
- private void removeSingleWorkerNode(String workerAddress) {
- workerNodeInfoWriteLock.lock();
- try {
- workerNodeInfo.remove(workerAddress);
- log.info("remove worker node {} from workerNodeInfo when worker server down", workerAddress);
- } finally {
- workerNodeInfoWriteLock.unlock();
- }
- }
- }
-
- class MasterDataListener implements SubscribeListener {
-
- @Override
- public void notify(Event event) {
- final String path = event.path();
- final Type type = event.type();
- if (registryClient.isMasterPath(path)) {
- try {
- if (type.equals(Type.ADD)) {
- log.info("master node : {} added.", path);
- } else if (type.equals(Type.REMOVE)) {
- log.info("master node : {} down.", path);
- alertDao.sendServerStoppedAlert(1, path, "MASTER");
- listenerEventAlertManager.publishServerDownListenerEvent(path, "MASTER");
- }
- } catch (Exception ex) {
- log.error("MasterNodeListener capture data change and get data failed.", ex);
- }
- }
- }
- }
-
- private void updateMasterNodes() {
- masterNodeInfoLock.lock();
- try {
- masterNodeInfo.clear();
- Map masterNodeMaps = registryClient.getServerMaps(RegistryNodeType.MASTER);
- for (Map.Entry entry : masterNodeMaps.entrySet()) {
- masterNodeInfo.put(entry.getKey(), JSONUtils.parseObject(entry.getValue(), MasterHeartBeat.class));
- }
- } catch (Exception e) {
- log.error("update master nodes error", e);
- } finally {
- masterNodeInfoLock.unlock();
- }
-
- }
-
- private void updateWorkerNodes() {
- workerNodeInfoWriteLock.lock();
- try {
- Map workerNodeMaps = registryClient.getServerMaps(RegistryNodeType.WORKER);
- for (Map.Entry entry : workerNodeMaps.entrySet()) {
- String nodeAddress = entry.getKey();
- WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(entry.getValue(), WorkerHeartBeat.class);
- workerNodeInfo.put(nodeAddress, workerHeartBeat);
- }
- } finally {
- workerNodeInfoWriteLock.unlock();
- }
- }
-
- private void updateWorkerGroupMappings() {
- List workerGroups = workerGroupMapper.queryAllWorkerGroup();
- Map> tmpWorkerGroupMappings = new HashMap<>();
- try {
- workerNodeInfoReadLock.lock();
- for (WorkerGroup workerGroup : workerGroups) {
- String workerGroupName = workerGroup.getName();
- String[] workerAddresses = workerGroup.getAddrList().split(Constants.COMMA);
- if (ArrayUtils.isEmpty(workerAddresses)) {
- continue;
- }
- Set activeWorkerNodes = Arrays.stream(workerAddresses)
- .filter(workerNodeInfo::containsKey).collect(Collectors.toSet());
- tmpWorkerGroupMappings.put(workerGroupName, activeWorkerNodes);
- }
- if (!tmpWorkerGroupMappings.containsKey(WorkerGroupUtils.getDefaultWorkerGroup())) {
- tmpWorkerGroupMappings.put(WorkerGroupUtils.getDefaultWorkerGroup(), workerNodeInfo.keySet());
- }
- } finally {
- workerNodeInfoReadLock.unlock();
- }
-
- workerGroupWriteLock.lock();
- try {
- workerGroupNodes.clear();
- workerGroupNodes.putAll(tmpWorkerGroupMappings);
- } finally {
- workerGroupWriteLock.unlock();
- }
- }
-
- public Map> getWorkerGroupNodes() {
- workerGroupReadLock.lock();
- try {
- return Collections.unmodifiableMap(workerGroupNodes);
- } finally {
- workerGroupReadLock.unlock();
- }
- }
-
- /**
- * get worker group nodes
- *
- * @param workerGroup workerGroup
- * @return worker nodes
- */
- public Set getWorkerGroupNodes(String workerGroup) throws WorkerGroupNotFoundException {
- workerGroupReadLock.lock();
- try {
- workerGroup = WorkerGroupUtils.getWorkerGroupOrDefault(workerGroup);
- Set nodes = workerGroupNodes.get(workerGroup);
- if (nodes == null) {
- throw new WorkerGroupNotFoundException(String.format("WorkerGroup: %s is invalidated", workerGroup));
- }
- if (CollectionUtils.isEmpty(nodes)) {
- return Collections.emptySet();
- }
- return Collections.unmodifiableSet(nodes);
- } finally {
- workerGroupReadLock.unlock();
- }
- }
-
- public Map getWorkerNodeInfo() {
- return Collections.unmodifiableMap(workerNodeInfo);
- }
-
- public Optional getWorkerNodeInfo(String workerServerAddress) {
- workerNodeInfoReadLock.lock();
- try {
- return Optional.ofNullable(workerNodeInfo.getOrDefault(workerServerAddress, null));
- } finally {
- workerNodeInfoReadLock.unlock();
- }
- }
-
- public Map getMasterNodeInfo() {
- return Collections.unmodifiableMap(masterNodeInfo);
- }
-
- /**
- * Add the resource change listener, when the resource changed, the listener will be notified.
- *
- * @param listener will be trigger, when the worker node info changed.
- */
- public synchronized void addWorkerInfoChangeListener(WorkerInfoChangeListener listener) {
- workerInfoChangeListeners.add(listener);
- }
-
- private void notifyWorkerInfoChangeListeners() {
- Map> workerGroupNodeMap = getWorkerGroupNodes();
- Map workerNodeInfoMap = getWorkerNodeInfo();
- for (WorkerInfoChangeListener listener : workerInfoChangeListeners) {
- listener.notify(workerGroupNodeMap, workerNodeInfoMap);
- }
- }
-
- public synchronized void addMasterInfoChangeListener(MasterInfoChangeListener listener) {
- masterInfoChangeListeners.add(listener);
- }
-
- private void notifyMasterInfoChangeListeners() {
- Map masterNodeInfoMap = getMasterNodeInfo();
- for (MasterInfoChangeListener listener : masterInfoChangeListeners) {
- listener.notify(masterNodeInfoMap);
- }
- }
-
- @PreDestroy
- public void destroy() {
- executorService.shutdownNow();
- }
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/WorkerInfoChangeListener.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/WorkerInfoChangeListener.java
deleted file mode 100644
index 9efc3517b8..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/WorkerInfoChangeListener.java
+++ /dev/null
@@ -1,38 +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.server.master.registry;
-
-import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
-
-import java.util.Map;
-import java.util.Set;
-
-/**
- * The listener used in {@link ServerNodeManager} to notify the change of worker info.
- */
-public interface WorkerInfoChangeListener {
-
- /**
- * Used to notify the change of worker info.
- *
- * @param workerGroups worker groups map, key is worker group name, value is worker address.
- * @param workerNodeInfo worker node info map, key is worker address, value is worker info.
- */
- void notify(Map> workerGroups, Map workerNodeInfo);
-
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java
deleted file mode 100644
index 793ae378d0..0000000000
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java
+++ /dev/null
@@ -1,102 +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.server.master.runner;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.dolphinscheduler.common.enums.TaskEventType;
-import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
-import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
-import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
-import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher;
-
-import java.util.Date;
-import java.util.Optional;
-
-import lombok.extern.slf4j.Slf4j;
-
-@Slf4j
-public abstract class BaseTaskDispatcher implements TaskDispatcher {
-
- protected final TaskEventService taskEventService;
- protected final MasterConfig masterConfig;
-
- protected BaseTaskDispatcher(TaskEventService taskEventService,
- MasterConfig masterConfig) {
- this.taskEventService = checkNotNull(taskEventService);
- this.masterConfig = checkNotNull(masterConfig);
- }
-
- @Override
- public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
- Host taskInstanceDispatchHost;
- try {
- taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable)
- .orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task."));
- } catch (WorkerGroupNotFoundException workerGroupNotFoundException) {
- // todo: this is a temporary solution, we should refactor the ServerNodeManager to make sure there won't
- // throw WorkerGroupNotFoundException unless the worker group is not exist in database
- throw new TaskDispatchException(
- "Dispatch task: " + taskExecuteRunnable.getTaskExecutionContext().getTaskName() + " failed",
- workerGroupNotFoundException);
- }
- taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress());
- doDispatch(taskExecuteRunnable);
- taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress());
- log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(),
- taskInstanceDispatchHost.getAddress());
- addDispatchEvent(taskExecuteRunnable);
- }
-
- protected abstract void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException;
-
- protected abstract Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException;
-
- protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) {
- TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
- TaskEvent taskEvent = TaskEvent.newDispatchEvent(
- taskExecutionContext.getProcessInstanceId(),
- taskExecutionContext.getTaskInstanceId(),
- taskExecutionContext.getHost());
- taskEventService.addEvent(taskEvent);
- }
-
- private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) {
- TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
- TaskEvent taskEvent = TaskEvent.builder()
- .processInstanceId(taskExecutionContext.getProcessInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .state(TaskExecutionStatus.FAILURE)
- .logPath(taskExecutionContext.getLogPath())
- .executePath(taskExecutionContext.getExecutePath())
- .appIds(taskExecutionContext.getAppIds())
- .processId(taskExecutionContext.getProcessId())
- .varPool(taskExecutionContext.getVarPool())
- .startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime()))
- .endTime(new Date())
- .event(TaskEventType.RESULT)
- .build();
- taskEventService.addEvent(taskEvent);
- }
-}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java
index 5cfb285c28..04ddae4769 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java
@@ -17,11 +17,18 @@
package org.apache.dolphinscheduler.server.master.runner;
+import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
+import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException;
+import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
+import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory;
+import java.util.Date;
import java.util.concurrent.atomic.AtomicBoolean;
import lombok.extern.slf4j.Slf4j;
@@ -39,6 +46,9 @@ public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread imple
@Autowired
private TaskDispatchFactory taskDispatchFactory;
+ @Autowired
+ private TaskEventService taskEventService;
+
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false);
public GlobalTaskDispatchWaitingQueueLooper() {
@@ -78,6 +88,11 @@ public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread imple
return;
}
taskDispatchFactory.getTaskDispatcher(taskInstance).dispatchTask(taskExecuteRunnable);
+ } catch (WorkerGroupNotFoundException workerGroupNotFoundException) {
+ // If the worker group not found then the task will not be dispatched anymore
+ log.error("Dispatch Task: {} failed, will send task failed event", taskInstance.getName(),
+ workerGroupNotFoundException);
+ addDispatchFailedEvent(taskExecuteRunnable);
} catch (Exception e) {
// If dispatch failed, will put the task back to the queue
// The task will be dispatched after waiting time.
@@ -98,4 +113,31 @@ public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread imple
log.error("GlobalTaskDispatchWaitingQueueLooper is not started");
}
}
+
+ private void addDispatchSuccessEvent(TaskExecuteRunnable taskExecuteRunnable) {
+ TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
+ TaskEvent taskEvent = TaskEvent.newDispatchEvent(
+ taskExecutionContext.getProcessInstanceId(),
+ taskExecutionContext.getTaskInstanceId(),
+ taskExecutionContext.getHost());
+ taskEventService.addEvent(taskEvent);
+ }
+
+ private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) {
+ TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
+ TaskEvent taskEvent = TaskEvent.builder()
+ .processInstanceId(taskExecutionContext.getProcessInstanceId())
+ .taskInstanceId(taskExecutionContext.getTaskInstanceId())
+ .state(TaskExecutionStatus.FAILURE)
+ .logPath(taskExecutionContext.getLogPath())
+ .executePath(taskExecutionContext.getExecutePath())
+ .appIds(taskExecutionContext.getAppIds())
+ .processId(taskExecutionContext.getProcessId())
+ .varPool(taskExecutionContext.getVarPool())
+ .startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime()))
+ .endTime(new Date())
+ .event(TaskEventType.RESULT)
+ .build();
+ taskEventService.addEvent(taskEvent);
+ }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java
index 89810aea1f..35d4244d18 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java
@@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.server.master.runner;
-import org.apache.dolphinscheduler.common.enums.SlotCheckState;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
@@ -25,7 +24,6 @@ import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.graph.IWorkflowGraph;
import org.apache.dolphinscheduler.server.master.graph.WorkflowGraphFactory;
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
-import org.apache.dolphinscheduler.server.master.registry.MasterSlotManager;
import org.apache.dolphinscheduler.service.exceptions.CronParseException;
import org.apache.dolphinscheduler.service.process.ProcessService;
@@ -40,9 +38,6 @@ import org.springframework.stereotype.Component;
@Component
public class WorkflowExecuteContextFactory {
- @Autowired
- private MasterSlotManager masterSlotManager;
-
@Autowired
private ProcessService processService;
@@ -69,33 +64,10 @@ public class WorkflowExecuteContextFactory {
private Optional createWorkflowInstance(Command command) throws CronParseException {
long commandTransformStartTime = System.currentTimeMillis();
- // Note: this check is not safe, the slot may change after command transform.
- // We use the database transaction in `handleCommand` so that we can guarantee the command will
- // always be executed
- // by only one master
- SlotCheckState slotCheckState = slotCheck(command);
- if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) {
- log.info("Master handle command {} skip, slot check state: {}", command.getId(), slotCheckState);
- throw new RuntimeException("Slot check failed the current state: " + slotCheckState);
- }
ProcessInstance processInstance = processService.handleCommand(masterConfig.getMasterAddress(), command);
ProcessInstanceMetrics
.recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime);
return Optional.ofNullable(processInstance);
}
- private SlotCheckState slotCheck(Command command) {
- int slot = masterSlotManager.getSlot();
- int masterSize = masterSlotManager.getMasterSize();
- SlotCheckState state;
- if (masterSize <= 0) {
- state = SlotCheckState.CHANGE;
- } else if (command.getId() % masterSize == slot) {
- state = SlotCheckState.PASS;
- } else {
- state = SlotCheckState.INJECT;
- }
- return state;
- }
-
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.java
new file mode 100644
index 0000000000..1b0fb6373d
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.java
@@ -0,0 +1,56 @@
+/*
+ * 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.server.master.runner.dispatcher;
+
+import org.apache.dolphinscheduler.extract.base.utils.Host;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException;
+import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable;
+
+import java.util.Optional;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public abstract class BaseTaskDispatcher implements TaskDispatcher {
+
+ @Override
+ public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
+ String taskName = taskExecuteRunnable.getTaskExecutionContext().getTaskName();
+ String taskInstanceDispatchAddress;
+ try {
+ taskInstanceDispatchAddress = getTaskInstanceDispatchHost(taskExecuteRunnable)
+ .map(Host::getAddress)
+ .orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task: " + taskName));
+ } catch (WorkerGroupNotFoundException workerGroupNotFoundException) {
+ // todo: this is a temporary solution, we should refactor the ServerNodeManager to make sure there won't
+ // throw WorkerGroupNotFoundException unless the worker group is not exist in database
+ throw new TaskDispatchException("Dispatch task: " + taskName + " failed", workerGroupNotFoundException);
+ }
+ taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchAddress);
+ // todo: add dispatch address here to avoid set host in TaskExecuteRunnable before
+ doDispatch(taskExecuteRunnable);
+ taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchAddress);
+ log.info("Success dispatch task {} to {}.", taskName, taskInstanceDispatchAddress);
+ }
+
+ protected abstract void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException;
+
+ protected abstract Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException;
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java
index cca7112975..bb73f967c6 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java
@@ -24,9 +24,7 @@ import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchR
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchResponse;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
-import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
-import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable;
import java.util.Optional;
@@ -41,10 +39,8 @@ public class MasterTaskDispatcher extends BaseTaskDispatcher {
private final Optional masterTaskExecuteHost;
- public MasterTaskDispatcher(TaskEventService taskEventService,
- MasterConfig masterConfig) {
- super(taskEventService, masterConfig);
- masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress()));
+ public MasterTaskDispatcher(MasterConfig masterConfig) {
+ this.masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress()));
}
@Override
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java
index f595d5a490..9ee683d97f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java
@@ -17,8 +17,8 @@
package org.apache.dolphinscheduler.server.master.runner.dispatcher;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable;
/**
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
index 3320082694..5488bee9fb 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
@@ -25,12 +25,8 @@ import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchResponse;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager;
-import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
-import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
-import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher;
+import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer;
+import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable;
import java.util.Optional;
@@ -43,39 +39,36 @@ import org.springframework.stereotype.Component;
@Component
public class WorkerTaskDispatcher extends BaseTaskDispatcher {
- private final HostManager hostManager;
+ private final IWorkerLoadBalancer workerLoadBalancer;
- public WorkerTaskDispatcher(TaskEventService taskEventService,
- MasterConfig masterConfig,
- HostManager hostManager) {
- super(taskEventService, masterConfig);
- this.hostManager = checkNotNull(hostManager);
+ public WorkerTaskDispatcher(IWorkerLoadBalancer workerLoadBalancer) {
+ this.workerLoadBalancer = checkNotNull(workerLoadBalancer);
}
@Override
protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
- TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
+ final TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
+ final String taskName = taskExecutionContext.getTaskName();
+ final String workerAddress = taskExecutionContext.getHost();
try {
ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
- .getProxyClient(taskExecutionContext.getHost(), ITaskInstanceOperator.class);
+ .getProxyClient(workerAddress, ITaskInstanceOperator.class);
TaskInstanceDispatchResponse taskInstanceDispatchResponse = taskInstanceOperator
.dispatchTask(new TaskInstanceDispatchRequest(taskExecuteRunnable.getTaskExecutionContext()));
if (!taskInstanceDispatchResponse.isDispatchSuccess()) {
- throw new TaskDispatchException(String.format("Dispatch task to %s failed, response is: %s",
- taskExecutionContext.getHost(), taskInstanceDispatchResponse));
+ throw new TaskDispatchException("Dispatch task: " + taskName + " to " + workerAddress + " failed: "
+ + taskInstanceDispatchResponse);
}
} catch (TaskDispatchException e) {
throw e;
} catch (Exception e) {
- throw new TaskDispatchException(String.format("Dispatch task to %s failed",
- taskExecutionContext.getHost()), e);
+ throw new TaskDispatchException("Dispatch task: " + taskName + " to " + workerAddress + " failed", e);
}
}
@Override
- protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) throws WorkerGroupNotFoundException {
+ protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) {
String workerGroup = taskExecuteRunnable.getTaskExecutionContext().getWorkerGroup();
- return hostManager.select(workerGroup);
-
+ return workerLoadBalancer.select(workerGroup).map(Host::of);
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/Selector.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactory.java
similarity index 70%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/Selector.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactory.java
index 49a4053e5e..fd4c9810ba 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/Selector.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactory.java
@@ -15,20 +15,15 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
+package org.apache.dolphinscheduler.server.master.utils;
-import java.util.Collection;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
-/**
- * selector
- * @param T
- */
-public interface Selector {
+public class MasterThreadFactory {
+
+ public static ScheduledExecutorService getDefaultSchedulerThreadExecutor() {
+ return Executors.newSingleThreadScheduledExecutor();
+ }
- /**
- * select
- * @param source source, the given source should not be empty.
- * @return T
- */
- T select(Collection source);
}
diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml
index da1e986928..56c2de6aa8 100644
--- a/dolphinscheduler-master/src/main/resources/application.yaml
+++ b/dolphinscheduler-master/src/main/resources/application.yaml
@@ -88,10 +88,6 @@ master:
pre-exec-threads: 10
# master execute thread number to limit process instances in parallel
exec-threads: 100
- # master dispatch task number per batch, if all the tasks dispatch failed in a batch, will sleep 1s.
- dispatch-task-number: 3
- # master host selector to select a suitable worker, default value: LowerWeight. Optional values include random, round_robin, lower_weight
- host-selector: lower_weight
# master heartbeat interval
max-heartbeat-interval: 10s
# master commit task retry times
@@ -127,6 +123,14 @@ master:
id-step: 1
# master fetch command num
fetch-size: 10
+ worker-load-balancer-configuration-properties:
+ # RANDOM, ROUND_ROBIN, FIXED_WEIGHTED_ROUND_ROBIN, DYNAMIC_WEIGHTED_ROUND_ROBIN
+ type: DYNAMIC_WEIGHTED_ROUND_ROBIN
+ # dynamic-weight-config-properties only used in DYNAMIC_WEIGHTED_ROUND_ROBIN, the weight of memory-usage, cpu-usage, task-thread-pool-usage should sum to 100.
+ dynamic-weight-config-properties:
+ memory-usage-weight: 30
+ cpu-usage-weight: 30
+ task-thread-pool-usage-weight: 40
server:
port: 5679
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterClustersTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterClustersTest.java
new file mode 100644
index 0000000000..d374cffcf9
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterClustersTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.server.master.cluster;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.jupiter.api.Test;
+
+class MasterClustersTest {
+
+ @Test
+ void getServers() {
+ MasterClusters masterClusters = new MasterClusters();
+ MasterServerMetadata normalMasterServerMetadata = getNormalMasterServerMetadata();
+ MasterServerMetadata busyMasterServerMetadata = getBusyMasterServerMetadata();
+ masterClusters.onServerAdded(normalMasterServerMetadata);
+ masterClusters.onServerAdded(busyMasterServerMetadata);
+ assertThat(masterClusters.getServers()).containsExactly(normalMasterServerMetadata, busyMasterServerMetadata);
+ }
+
+ @Test
+ void getNormalServers() {
+ MasterClusters masterClusters = new MasterClusters();
+ MasterServerMetadata normalMasterServerMetadata = getNormalMasterServerMetadata();
+ MasterServerMetadata busyMasterServerMetadata = getBusyMasterServerMetadata();
+ masterClusters.onServerAdded(normalMasterServerMetadata);
+ masterClusters.onServerAdded(busyMasterServerMetadata);
+ assertThat(masterClusters.getNormalServers()).containsExactly(normalMasterServerMetadata);
+ }
+
+ @Test
+ void registerListener() {
+ MasterClusters masterClusters = new MasterClusters();
+ AtomicBoolean addServerFlag = new AtomicBoolean(false);
+ masterClusters.registerListener(
+ (IClusters.ServerAddedListener) server -> addServerFlag.set(true));
+ masterClusters.onServerAdded(getNormalMasterServerMetadata());
+ assertThat(addServerFlag.get()).isTrue();
+ }
+
+ @Test
+ void onServerAdded() {
+ MasterClusters masterClusters = new MasterClusters();
+ masterClusters.onServerAdded(getNormalMasterServerMetadata());
+ assertThat(masterClusters.getServers()).containsExactly(getNormalMasterServerMetadata());
+ }
+
+ @Test
+ void onServerRemove() {
+ MasterClusters masterClusters = new MasterClusters();
+ masterClusters.onServerAdded(getNormalMasterServerMetadata());
+ masterClusters.onServerRemove(getNormalMasterServerMetadata());
+ assertThat(masterClusters.getServers()).isEmpty();
+ }
+
+ @Test
+ void onServerUpdate() {
+ MasterClusters masterClusters = new MasterClusters();
+ masterClusters.onServerAdded(getNormalMasterServerMetadata());
+ MasterServerMetadata updatedMasterServerMetadata = MasterServerMetadata.builder()
+ .address("127.0.0.1:5679")
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.BUSY)
+ .build();
+ masterClusters.onServerUpdate(updatedMasterServerMetadata);
+ assertThat(masterClusters.getServers()).containsExactly(updatedMasterServerMetadata);
+ }
+
+ private MasterServerMetadata getNormalMasterServerMetadata() {
+ return MasterServerMetadata.builder()
+ .address("127.0.0.1:5679")
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .build();
+ }
+
+ private MasterServerMetadata getBusyMasterServerMetadata() {
+ return MasterServerMetadata.builder()
+ .address("127.0.0.2:1235")
+ .cpuUsage(0.8)
+ .memoryUsage(0.8)
+ .serverStatus(ServerStatus.BUSY)
+ .build();
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadataTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadataTest.java
new file mode 100644
index 0000000000..241d026010
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadataTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.server.master.cluster;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.common.model.MasterHeartBeat;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.truth.Truth;
+
+class MasterServerMetadataTest {
+
+ @Test
+ void parseFromHeartBeat() {
+ MasterHeartBeat masterHeartBeat = MasterHeartBeat.builder()
+ .processId(30001)
+ .startupTime(System.currentTimeMillis())
+ .reportTime(System.currentTimeMillis())
+ .jvmCpuUsage(0.1)
+ .cpuUsage(0.2)
+ .jvmMemoryUsage(0.3)
+ .memoryUsage(0.4)
+ .diskUsage(0.5)
+ .serverStatus(ServerStatus.NORMAL)
+ .host("localhost")
+ .port(12345)
+ .build();
+ MasterServerMetadata masterServer = MasterServerMetadata.parseFromHeartBeat(masterHeartBeat);
+ Truth.assertThat(masterServer.getCpuUsage()).isEqualTo(0.2);
+ Truth.assertThat(masterServer.getMemoryUsage()).isEqualTo(0.4);
+ Truth.assertThat(masterServer.getServerStatus()).isEqualTo(ServerStatus.NORMAL);
+ Truth.assertThat(masterServer.getAddress()).isEqualTo("localhost:12345");
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java
new file mode 100644
index 0000000000..56d119d776
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.server.master.cluster;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class MasterSlotManagerTest {
+
+ private MasterSlotManager masterSlotManager;
+
+ private ClusterManager clusterManager;
+
+ private MasterConfig masterConfig;
+
+ @BeforeEach
+ public void setUp() {
+ clusterManager = new ClusterManager();
+ masterConfig = new MasterConfig();
+ masterConfig.setMasterAddress("127.0.0.1:5678");
+ masterSlotManager = new MasterSlotManager(clusterManager, masterConfig);
+ MasterServerMetadata master1 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .address(masterConfig.getMasterAddress())
+ .build();
+ MasterServerMetadata master2 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .address("127.0.0.2:5679")
+ .build();
+ MasterServerMetadata master3 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .address("127.0.0.3:5679")
+ .build();
+ MasterServerMetadata master4 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.BUSY)
+ .address("127.0.0.4:5679")
+ .build();
+ clusterManager.getMasterClusters().onServerAdded(master1);
+ clusterManager.getMasterClusters().onServerAdded(master2);
+ clusterManager.getMasterClusters().onServerAdded(master3);
+ clusterManager.getMasterClusters().onServerAdded(master4);
+ }
+
+ @Test
+ void getCurrentMasterSlot() {
+ assertThat(masterSlotManager.getCurrentMasterSlot()).isAtLeast(0);
+ }
+
+ @Test
+ void getTotalMasterSlots() {
+ assertThat(masterSlotManager.getTotalMasterSlots()).isEqualTo(3);
+ }
+
+ @Test
+ void checkSlotValid() {
+ assertThat(masterSlotManager.checkSlotValid()).isTrue();
+ }
+
+ @Test
+ void doReBalance() {
+ MasterServerMetadata master2 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .address("127.0.0.3:5679")
+ .build();
+ MasterServerMetadata master3 = MasterServerMetadata.builder()
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.BUSY)
+ .address("127.0.0.4:5679")
+ .build();
+ clusterManager.getMasterClusters().onServerRemove(master2);
+ clusterManager.getMasterClusters().onServerRemove(master3);
+ // After doReBalance, the total master slots should be 2
+ assertThat(masterSlotManager.getTotalMasterSlots()).isEqualTo(2);
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java
new file mode 100644
index 0000000000..e2106ebd23
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.server.master.cluster;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.truth.Truth;
+
+class WorkerClustersTest {
+
+ @Test
+ void testOnWorkerGroupDelete() {
+ WorkerClusters workerClusters = new WorkerClusters();
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+
+ WorkerGroup workerGroup = WorkerGroup.builder()
+ .name("flinkCluster")
+ .addrList(normalWorkerServerMetadata.getAddress())
+ .build();
+ workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup));
+ assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+
+ workerClusters.onWorkerGroupDelete(Lists.newArrayList(workerGroup));
+ Truth.assertThat(workerClusters.containsWorkerGroup("flinkCluster")).isFalse();
+ assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")).isEmpty();
+ }
+
+ @Test
+ void testOnWorkerGroupAdd() {
+ WorkerClusters workerClusters = new WorkerClusters();
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+
+ WorkerGroup workerGroup = WorkerGroup.builder()
+ .name("flinkCluster")
+ .addrList(normalWorkerServerMetadata.getAddress())
+ .build();
+ workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup));
+ assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+ }
+
+ @Test
+ void testOnWorkerGroupChange() {
+ WorkerClusters workerClusters = new WorkerClusters();
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+
+ WorkerGroup workerGroup = WorkerGroup.builder()
+ .name("flinkCluster")
+ .addrList(normalWorkerServerMetadata.getAddress())
+ .build();
+ workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup));
+ assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+
+ WorkerGroup updatedWorkerGroup = WorkerGroup.builder()
+ .name("flinkCluster")
+ .addrList("")
+ .build();
+ workerClusters.onWorkerGroupChange(Lists.newArrayList(updatedWorkerGroup));
+ assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")).isEmpty();
+ assertThat(workerClusters.containsWorkerGroup("flinkCluster")).isTrue();
+ }
+
+ @Test
+ void testOnServerAdded() {
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata();
+
+ WorkerClusters workerClusters = new WorkerClusters();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+ workerClusters.onServerAdded(busyWorkerServerMetadata);
+ assertThat(workerClusters.getWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress(), busyWorkerServerMetadata.getAddress());
+ assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+ }
+
+ @Test
+ void testOnServerRemove() {
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata();
+
+ WorkerClusters workerClusters = new WorkerClusters();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+ workerClusters.onServerAdded(busyWorkerServerMetadata);
+ workerClusters.onServerRemove(busyWorkerServerMetadata);
+
+ assertThat(workerClusters.getWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+ assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress());
+ }
+
+ @Test
+ void testOnServerUpdate() {
+
+ WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata();
+ WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata();
+
+ WorkerClusters workerClusters = new WorkerClusters();
+ workerClusters.onServerAdded(normalWorkerServerMetadata);
+ workerClusters.onServerAdded(busyWorkerServerMetadata);
+
+ WorkerServerMetadata workerServerMetadata = WorkerServerMetadata.builder()
+ .address("127.0.0.2:1235")
+ .cpuUsage(0.3)
+ .memoryUsage(0.3)
+ .serverStatus(ServerStatus.NORMAL)
+ .taskThreadPoolUsage(0.3)
+ .workerWeight(2)
+ .build();
+
+ workerClusters.onServerUpdate(workerServerMetadata);
+
+ assertThat(workerClusters.getWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress(), workerServerMetadata.getAddress());
+ assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default"))
+ .containsExactly(normalWorkerServerMetadata.getAddress(), workerServerMetadata.getAddress());
+ assertThat(workerClusters.getServers()).containsExactly(normalWorkerServerMetadata, workerServerMetadata);
+ }
+
+ private WorkerServerMetadata getNormalWorkerServerMetadata() {
+ return WorkerServerMetadata.builder()
+ .address("127.0.0.1:1235")
+ .cpuUsage(0.2)
+ .memoryUsage(0.4)
+ .serverStatus(ServerStatus.NORMAL)
+ .taskThreadPoolUsage(0.6)
+ .workerWeight(2)
+ .build();
+ }
+
+ private WorkerServerMetadata getBusyWorkerServerMetadata() {
+ return WorkerServerMetadata.builder()
+ .address("127.0.0.2:1235")
+ .cpuUsage(0.8)
+ .memoryUsage(0.8)
+ .serverStatus(ServerStatus.BUSY)
+ .taskThreadPoolUsage(0.9)
+ .workerWeight(2)
+ .build();
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifierTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifierTest.java
new file mode 100644
index 0000000000..0bc7d70fd5
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifierTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.server.master.cluster;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
+import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+
+class WorkerGroupChangeNotifierTest {
+
+ @Test
+ void detectWorkerGroupChanges_addedWorkerGroup() {
+ WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class);
+ WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao);
+
+ WorkerGroup workerGroup1 = WorkerGroup.builder()
+ .name("workerGroup1")
+ .addrList("")
+ .build();
+ when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1));
+
+ AtomicBoolean workerGroupAdded = new AtomicBoolean(false);
+ AtomicBoolean workerGroupDeleted = new AtomicBoolean(false);
+ AtomicBoolean workerGroupChanged = new AtomicBoolean(false);
+ workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() {
+
+ @Override
+ public void onWorkerGroupDelete(List workerGroups) {
+ workerGroupDeleted.set(true);
+ }
+
+ @Override
+ public void onWorkerGroupAdd(List workerGroups) {
+ workerGroupAdded.set(true);
+ assertThat(workerGroups).containsExactly(workerGroup1);
+ }
+
+ @Override
+ public void onWorkerGroupChange(List workerGroups) {
+ workerGroupChanged.set(true);
+ }
+ });
+ workerGroupChangeNotifier.detectWorkerGroupChanges();
+ assertThat(workerGroupAdded.get()).isTrue();
+ assertThat(workerGroupChanged.get()).isFalse();
+ assertThat(workerGroupDeleted.get()).isFalse();
+ }
+
+ @Test
+ void detectWorkerGroupChanges_deleteWorkerGroup() {
+ WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class);
+ WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao);
+
+ WorkerGroup workerGroup1 = WorkerGroup.builder()
+ .name("workerGroup1")
+ .addrList("")
+ .build();
+ when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1));
+ workerGroupChangeNotifier.detectWorkerGroupChanges();
+
+ when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList());
+ AtomicBoolean workerGroupAdded = new AtomicBoolean(false);
+ AtomicBoolean workerGroupDeleted = new AtomicBoolean(false);
+ AtomicBoolean workerGroupChanged = new AtomicBoolean(false);
+ workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() {
+
+ @Override
+ public void onWorkerGroupDelete(List workerGroups) {
+ workerGroupDeleted.set(true);
+ assertThat(workerGroups).containsExactly(workerGroup1);
+ }
+
+ @Override
+ public void onWorkerGroupAdd(List workerGroups) {
+ workerGroupAdded.set(true);
+ }
+
+ @Override
+ public void onWorkerGroupChange(List workerGroups) {
+ workerGroupChanged.set(true);
+ }
+ });
+ workerGroupChangeNotifier.detectWorkerGroupChanges();
+ assertThat(workerGroupAdded.get()).isFalse();
+ assertThat(workerGroupChanged.get()).isFalse();
+ assertThat(workerGroupDeleted.get()).isTrue();
+ assertThat(workerGroupChangeNotifier.getWorkerGroupMap()).isEmpty();
+ }
+
+ @Test
+ void detectWorkerGroupChanges_updateWorkerGroup() {
+ WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class);
+ WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao);
+
+ WorkerGroup workerGroup1 = WorkerGroup.builder()
+ .name("workerGroup1")
+ .addrList("")
+ .build();
+ when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1));
+ workerGroupChangeNotifier.detectWorkerGroupChanges();
+
+ WorkerGroup updatedWorkerGroup1 = WorkerGroup.builder()
+ .name("workerGroup1")
+ .addrList("127.0.0.1:1235")
+ .build();
+ when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(updatedWorkerGroup1));
+ AtomicBoolean workerGroupAdded = new AtomicBoolean(false);
+ AtomicBoolean workerGroupDeleted = new AtomicBoolean(false);
+ AtomicBoolean workerGroupChanged = new AtomicBoolean(false);
+ workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() {
+
+ @Override
+ public void onWorkerGroupDelete(List workerGroups) {
+ workerGroupDeleted.set(true);
+ }
+
+ @Override
+ public void onWorkerGroupAdd(List workerGroups) {
+ workerGroupAdded.set(true);
+ }
+
+ @Override
+ public void onWorkerGroupChange(List workerGroups) {
+ workerGroupChanged.set(true);
+ assertThat(workerGroups).containsExactly(updatedWorkerGroup1);
+ }
+ });
+ workerGroupChangeNotifier.detectWorkerGroupChanges();
+ assertThat(workerGroupAdded.get()).isFalse();
+ assertThat(workerGroupChanged.get()).isTrue();
+ assertThat(workerGroupDeleted.get()).isFalse();
+ assertThat(workerGroupChangeNotifier.getWorkerGroupMap()).containsEntry("workerGroup1", updatedWorkerGroup1);
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java
new file mode 100644
index 0000000000..aef6bf9ee0
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.server.master.cluster;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.truth.Truth;
+
+class WorkerServerMetadataTest {
+
+ @Test
+ void parseFromHeartBeat() {
+ WorkerHeartBeat workerHeartBeat = WorkerHeartBeat.builder()
+ .processId(30001)
+ .startupTime(System.currentTimeMillis())
+ .reportTime(System.currentTimeMillis())
+ .jvmCpuUsage(0.1)
+ .cpuUsage(0.2)
+ .jvmMemoryUsage(0.3)
+ .memoryUsage(0.4)
+ .diskUsage(0.5)
+ .serverStatus(ServerStatus.NORMAL)
+ .host("localhost")
+ .port(12345)
+ .workerHostWeight(2)
+ .threadPoolUsage(0.6)
+ .build();
+ WorkerServerMetadata workerServerMetadata = WorkerServerMetadata.parseFromHeartBeat(workerHeartBeat);
+ Truth.assertThat(workerServerMetadata.getCpuUsage()).isEqualTo(0.2);
+ Truth.assertThat(workerServerMetadata.getMemoryUsage()).isEqualTo(0.4);
+ Truth.assertThat(workerServerMetadata.getServerStatus()).isEqualTo(ServerStatus.NORMAL);
+ Truth.assertThat(workerServerMetadata.getAddress()).isEqualTo("localhost:12345");
+ Truth.assertThat(workerServerMetadata.getWorkerWeight()).isEqualTo(2);
+ Truth.assertThat(workerServerMetadata.getTaskThreadPoolUsage()).isEqualTo(0.6);
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java
new file mode 100644
index 0000000000..a60cf236da
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+public class BaseWorkerLoadBalancerTest {
+
+ /**
+ * "default" -> 127.0.0.1:1234, 127.0.0.2:1234, 127.0.0.3:1234, 127.0.0.4:1234
+ * "sparkCluster" -> 127.0.0.1:1234
+ * "flinkCluster" ->
+ * "busyCluster" -> 127.0.0.4:1234
+ */
+ protected WorkerClusters getDefaultWorkerClusters() {
+ List workerGroups = Lists.newArrayList(
+ WorkerGroup.builder()
+ .name("sparkCluster")
+ .addrList("127.0.0.1:1234")
+ .build(),
+ WorkerGroup.builder()
+ .name("flinkCluster")
+ .build(),
+ WorkerGroup.builder()
+ .name("busyCluster")
+ .build());
+
+ List workerServers = Lists.newArrayList(
+ WorkerServerMetadata.builder()
+ .address("127.0.0.1:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.1)
+ .memoryUsage(0.1)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.2:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.2)
+ .memoryUsage(0.2)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.3:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.3)
+ .memoryUsage(0.3)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.4:1234")
+ .serverStatus(ServerStatus.BUSY)
+ .cpuUsage(0.8)
+ .memoryUsage(0.8)
+ .build());
+
+ WorkerClusters workerClusters = new WorkerClusters();
+ workerClusters.onWorkerGroupAdd(workerGroups);
+ workerServers.forEach(workerClusters::onServerAdded);
+ return workerClusters;
+ }
+
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java
new file mode 100644
index 0000000000..fa870ee8d9
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.truth.Truth;
+
+class DynamicWeightedRoundRobinWorkerLoadBalancerTest {
+
+ @Test
+ void select_defaultWorkerGroup() {
+ DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer();
+
+ // select 10000 times to check if the selected worker is in the worker list
+ Map workerSelectedCount = new HashMap<>();
+ for (int i = 0; i < 10000; i++) {
+ Optional selectedWorker = loadBalancer.select("default");
+ Truth.assertThat(selectedWorker).isPresent();
+ workerSelectedCount.put(selectedWorker.get(),
+ workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1);
+ }
+ // Assert the three workers selected times are similar
+ Integer times1 = workerSelectedCount.get("127.0.0.1:1234");
+ Integer times2 = workerSelectedCount.get("127.0.0.2:1234");
+ Integer times3 = workerSelectedCount.get("127.0.0.3:1234");
+ Truth.assertThat(Math.abs(times1 - times2 * 2)).isWithin(10);
+ Truth.assertThat(Math.abs(times1 - times3 * 3)).isWithin(10);
+ Truth.assertThat(times1 + times2 + times3).isEqualTo(10000);
+ }
+
+ @Test
+ void select_emptyWorkerGroup() {
+ DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer();
+ Truth.assertThat(loadBalancer.select("busyCluster")).isEmpty();
+ }
+
+ @Test
+ void select_workerGroupNotExist() {
+ DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer();
+ Truth.assertThat(loadBalancer.select("notExistCluster")).isEmpty();
+ }
+
+ private DynamicWeightedRoundRobinWorkerLoadBalancer createDynamicWeightedRoundRobinWorkerLoadBalancer() {
+ List workerGroups = Lists.newArrayList(
+ WorkerGroup.builder()
+ .name("sparkCluster")
+ .addrList("127.0.0.1:1234")
+ .build(),
+ WorkerGroup.builder()
+ .name("flinkCluster")
+ .build(),
+ WorkerGroup.builder()
+ .name("busyCluster")
+ .build());
+ List workerServers = Lists.newArrayList(
+ WorkerServerMetadata.builder()
+ .address("127.0.0.1:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.1)
+ .memoryUsage(0.1)
+ .taskThreadPoolUsage(0.1)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.2:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.2)
+ .memoryUsage(0.2)
+ .taskThreadPoolUsage(0.2)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.3:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.3)
+ .memoryUsage(0.3)
+ .taskThreadPoolUsage(0.3)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.4:1234")
+ .serverStatus(ServerStatus.BUSY)
+ .cpuUsage(0.8)
+ .memoryUsage(0.8)
+ .build());
+ WorkerClusters workerClusters = new WorkerClusters();
+ DynamicWeightedRoundRobinWorkerLoadBalancer dynamicWeightedRoundRobinWorkerLoadBalancer =
+ new DynamicWeightedRoundRobinWorkerLoadBalancer(workerClusters,
+ new WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties());
+ workerClusters.onWorkerGroupChange(workerGroups);
+ workerServers.forEach(workerClusters::onServerAdded);
+
+ return dynamicWeightedRoundRobinWorkerLoadBalancer;
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java
new file mode 100644
index 0000000000..6e27098026
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.common.enums.ServerStatus;
+import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.truth.Truth;
+
+class FixedWeightedRoundRobinWorkerLoadBalancerTest {
+
+ @Test
+ void select_defaultWorkerGroup() {
+ FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer();
+
+ // select 10000 times to check if the selected worker is in the worker list
+ Map workerSelectedCount = new HashMap<>();
+ for (int i = 0; i < 10000; i++) {
+ Optional selectedWorker = loadBalancer.select("default");
+ Truth.assertThat(selectedWorker).isPresent();
+ workerSelectedCount.put(selectedWorker.get(),
+ workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1);
+ }
+ // Assert the three workers selected times are similar
+ Integer times1 = workerSelectedCount.get("127.0.0.1:1234");
+ Integer times2 = workerSelectedCount.get("127.0.0.2:1234");
+ Integer times3 = workerSelectedCount.get("127.0.0.3:1234");
+ Truth.assertThat(Math.abs(times2 - times1 * 2)).isWithin(10);
+ Truth.assertThat(Math.abs(times3 - times1 * 3)).isWithin(10);
+ Truth.assertThat(times1 + times2 + times3).isEqualTo(10000);
+ }
+
+ @Test
+ void select_emptyWorkerGroup() {
+ FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer();
+ Truth.assertThat(loadBalancer.select("busyCluster")).isEmpty();
+ }
+
+ @Test
+ void select_workerGroupNotExist() {
+ FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer();
+ Truth.assertThat(loadBalancer.select("notExistCluster")).isEmpty();
+ }
+
+ private FixedWeightedRoundRobinWorkerLoadBalancer createFixedWeightedRoundRobinWorkerLoadBalancer() {
+ List workerGroups = Lists.newArrayList(
+ WorkerGroup.builder()
+ .name("sparkCluster")
+ .addrList("127.0.0.1:1234")
+ .build(),
+ WorkerGroup.builder()
+ .name("flinkCluster")
+ .build(),
+ WorkerGroup.builder()
+ .name("busyCluster")
+ .build());
+ List workerServers = Lists.newArrayList(
+ WorkerServerMetadata.builder()
+ .address("127.0.0.1:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.1)
+ .memoryUsage(0.1)
+ .workerWeight(1)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.2:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.2)
+ .memoryUsage(0.2)
+ .workerWeight(2)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.3:1234")
+ .serverStatus(ServerStatus.NORMAL)
+ .cpuUsage(0.3)
+ .memoryUsage(0.3)
+ .workerWeight(3)
+ .build(),
+ WorkerServerMetadata.builder()
+ .address("127.0.0.4:1234")
+ .serverStatus(ServerStatus.BUSY)
+ .cpuUsage(0.8)
+ .memoryUsage(0.8)
+ .build());
+ WorkerClusters workerClusters = new WorkerClusters();
+ FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer =
+ new FixedWeightedRoundRobinWorkerLoadBalancer(workerClusters);
+ workerClusters.onWorkerGroupChange(workerGroups);
+ workerServers.forEach(workerClusters::onServerAdded);
+
+ return loadBalancer;
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancerTest.java
new file mode 100644
index 0000000000..f45fd24af6
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RandomWorkerLoadBalancerTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.truth.Truth;
+
+class RandomWorkerLoadBalancerTest extends BaseWorkerLoadBalancerTest {
+
+ @Test
+ void select_defaultWorkerGroup() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters);
+ // select 10000 times to check if the selected worker is in the worker list
+ Map workerSelectedCount = new HashMap<>();
+ for (int i = 0; i < 10000; i++) {
+ Optional selectedWorker = randomWorkerLoadBalancer.select("default");
+ Truth.assertThat(selectedWorker).isPresent();
+ workerSelectedCount.put(selectedWorker.get(),
+ workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1);
+ }
+ // Assert the three workers selected times are similar
+ Integer times1 = workerSelectedCount.get("127.0.0.1:1234");
+ Integer times2 = workerSelectedCount.get("127.0.0.2:1234");
+ Integer times3 = workerSelectedCount.get("127.0.0.3:1234");
+ Truth.assertThat(Math.abs(times1 - times2)).isWithin(10);
+ Truth.assertThat(Math.abs(times1 - times3)).isWithin(10);
+ Truth.assertThat(Math.abs(times2 - times3)).isWithin(10);
+ }
+
+ @Test
+ void select_emptyWorkerGroup() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters);
+ Truth.assertThat(randomWorkerLoadBalancer.select("busyCluster")).isEmpty();
+ }
+
+ @Test
+ void select_workerGroupNotExist() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters);
+ Truth.assertThat(randomWorkerLoadBalancer.select("notExistCluster")).isEmpty();
+ }
+
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancerTest.java
new file mode 100644
index 0000000000..a8c17340ad
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/RoundRobinWorkerLoadBalancerTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.server.master.cluster.loadbalancer;
+
+import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import org.junit.jupiter.api.Test;
+
+import com.google.common.truth.Truth;
+
+class RoundRobinWorkerLoadBalancerTest extends BaseWorkerLoadBalancerTest {
+
+ @Test
+ void select_defaultWorkerGroup() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer =
+ new RoundRobinWorkerLoadBalancer(defaultWorkerClusters);
+ // select 10000 times to check if the selected worker is in the worker list
+ Map workerSelectedCount = new HashMap<>();
+ for (int i = 0; i < 10000; i++) {
+ Optional selectedWorker = roundRobinWorkerLoadBalancer.select("default");
+ Truth.assertThat(selectedWorker).isPresent();
+ workerSelectedCount.put(selectedWorker.get(),
+ workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1);
+ }
+ // Assert the three workers selected times are similar
+ Integer times1 = workerSelectedCount.get("127.0.0.1:1234");
+ Integer times2 = workerSelectedCount.get("127.0.0.2:1234");
+ Integer times3 = workerSelectedCount.get("127.0.0.3:1234");
+ Truth.assertThat(Math.abs(times1 - times2)).isWithin(10);
+ Truth.assertThat(Math.abs(times1 - times3)).isWithin(10);
+ Truth.assertThat(Math.abs(times2 - times3)).isWithin(10);
+ }
+
+ @Test
+ void select_emptyWorkerGroup() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer =
+ new RoundRobinWorkerLoadBalancer(defaultWorkerClusters);
+ Truth.assertThat(roundRobinWorkerLoadBalancer.select("busyCluster")).isEmpty();
+ }
+
+ @Test
+ void select_workerGroupNotExist() {
+ WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters();
+ RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer =
+ new RoundRobinWorkerLoadBalancer(defaultWorkerClusters);
+ Truth.assertThat(roundRobinWorkerLoadBalancer.select("notExistCluster")).isEmpty();
+ }
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/config/MasterConfigTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/config/MasterConfigTest.java
index 9d26aa81f4..b767423946 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/config/MasterConfigTest.java
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/config/MasterConfigTest.java
@@ -21,6 +21,9 @@ import static com.google.common.truth.Truth.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.WorkerLoadBalancerConfigurationProperties;
+import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.WorkerLoadBalancerType;
+
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.boot.test.autoconfigure.web.servlet.AutoConfigureMockMvc;
@@ -33,12 +36,6 @@ public class MasterConfigTest {
@Autowired
private MasterConfig masterConfig;
- @Test
- public void getMasterDispatchTaskNumber() {
- int masterDispatchTaskNumber = masterConfig.getDispatchTaskNumber();
- assertEquals(30, masterDispatchTaskNumber);
- }
-
@Test
public void getServerLoadProtection() {
MasterServerLoadProtection serverLoadProtection = masterConfig.getServerLoadProtection();
@@ -61,4 +58,18 @@ public class MasterConfigTest {
assertThat(idSlotBasedFetchConfig.getIdStep()).isEqualTo(3);
assertThat(idSlotBasedFetchConfig.getFetchSize()).isEqualTo(11);
}
+
+ @Test
+ public void getWorkerLoadBalancerConfigurationProperties() {
+ WorkerLoadBalancerConfigurationProperties workerLoadBalancerConfigurationProperties =
+ masterConfig.getWorkerLoadBalancerConfigurationProperties();
+ assertThat(workerLoadBalancerConfigurationProperties.getType())
+ .isEqualTo(WorkerLoadBalancerType.DYNAMIC_WEIGHTED_ROUND_ROBIN);
+
+ WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties dynamicWeightConfigProperties =
+ workerLoadBalancerConfigurationProperties.getDynamicWeightConfigProperties();
+ assertThat(dynamicWeightConfigProperties.getMemoryUsageWeight()).isEqualTo(40);
+ assertThat(dynamicWeightConfigProperties.getCpuUsageWeight()).isEqualTo(30);
+ assertThat(dynamicWeightConfigProperties.getTaskThreadPoolUsageWeight()).isEqualTo(30);
+ }
}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorkerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorkerTest.java
deleted file mode 100644
index 352cb869dc..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWorkerTest.java
+++ /dev/null
@@ -1,43 +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.server.master.dispatch.host.assign;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-public class HostWorkerTest {
-
- @Test
- public void testHostWorker1() {
- HostWorker hostWorker = new HostWorker("192.158.2.2", 11, 20, "default");
- Assertions.assertEquals("192.158.2.2", hostWorker.getIp());
- Assertions.assertEquals(11, hostWorker.getPort());
- Assertions.assertEquals(20, hostWorker.getHostWeight());
- Assertions.assertEquals("default", hostWorker.getWorkerGroup());
- }
-
- @Test
- public void testHostWorker2() {
- HostWorker hostWorker = HostWorker.of("192.158.2.2:22", 80, "default");
- Assertions.assertEquals("192.158.2.2", hostWorker.getIp());
- Assertions.assertEquals(22, hostWorker.getPort());
- Assertions.assertEquals(80, hostWorker.getHostWeight());
- Assertions.assertEquals("default", hostWorker.getWorkerGroup());
- }
-
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
deleted file mode 100644
index c8b4c71fc0..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
+++ /dev/null
@@ -1,97 +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.server.master.dispatch.host.assign;
-
-import java.util.ArrayList;
-import java.util.Collection;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-public class LowerWeightRoundRobinTest {
-
- @Test
- public void testSelect() {
- Collection sources = new ArrayList<>();
- sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.384, 0.1,
- System.currentTimeMillis() - 60 * 8 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.56, 0.324, 0.2,
- System.currentTimeMillis() - 60 * 5 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.80, 0.315, 0.1,
- System.currentTimeMillis() - 60 * 2 * 1000));
-
- LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
- HostWeight result;
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.2", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.2", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.2", result.getHost().getIp());
- }
-
- @Test
- public void testWarmUpSelect() {
- Collection sources = new ArrayList<>();
- sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.384, 0,
- System.currentTimeMillis() - 60 * 8 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.44, 0.384, 0,
- System.currentTimeMillis() - 60 * 5 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.44, 0.384, 0,
- System.currentTimeMillis() - 60 * 3 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.4:33", 100, "default"), 0.06, 0.44, 0.384, 0,
- System.currentTimeMillis() - 60 * 11 * 1000));
-
- LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
- HostWeight result;
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.4", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.2", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.4", result.getHost().getIp());
- result = roundRobin.select(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- }
-
- @Test
- public void testDoSelect() {
- Collection sources = new ArrayList<>();
- LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
- HostWeight result;
- result = roundRobin.doSelect(sources);
- Assertions.assertNull(result);
-
- sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.314, 0.1,
- System.currentTimeMillis() - 60 * 8 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.56, 0.324, 0.2,
- System.currentTimeMillis() - 60 * 5 * 1000));
- sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.80, 0.315, 0.1,
- System.currentTimeMillis() - 60 * 2 * 1000));
- result = roundRobin.doSelect(sources);
- Assertions.assertEquals("192.158.2.1", result.getHost().getIp());
- }
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
deleted file mode 100644
index baad4f51f4..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
+++ /dev/null
@@ -1,53 +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.server.master.dispatch.host.assign;
-
-import java.util.Arrays;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-/**
- * random selector
- */
-public class RandomSelectorTest {
-
- @Test
- public void testSelectWithIllegalArgumentException() {
- RandomSelector selector = new RandomSelector();
- Assertions.assertThrows(IllegalArgumentException.class, () -> {
- selector.select(null);
- });
- }
-
- @Test
- public void testSelect1() {
- RandomSelector selector = new RandomSelector();
- HostWorker result = selector.select(Arrays.asList(new HostWorker("192.168.1.1:11", 100, "default"),
- new HostWorker("192.168.1.2:22", 80, "default")));
- Assertions.assertNotNull(result);
- }
-
- @Test
- public void testSelect() {
- RandomSelector selector = new RandomSelector();
- HostWorker result = selector.select(Arrays.asList(new HostWorker("192.168.1.1", 11, 100, "default"),
- new HostWorker("192.168.1.2:", 22, 20, "default")));
- Assertions.assertNotNull(result);
- }
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
deleted file mode 100644
index 910998abf7..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
+++ /dev/null
@@ -1,106 +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.server.master.dispatch.host.assign;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-/**
- * round robin selector
- */
-public class RoundRobinSelectorTest {
-
- @Test
- public void testSelectWithIllegalArgumentException() {
- RoundRobinSelector selector = new RoundRobinSelector();
- Assertions.assertThrows(IllegalArgumentException.class, () -> {
- selector.select(null);
- });
- }
-
- @Test
- public void testSelect1() {
- RoundRobinSelector selector = new RoundRobinSelector();
- List hostOneList = Arrays.asList(
- new HostWorker("192.168.1.1", 80, 20, "kris"),
- new HostWorker("192.168.1.2", 80, 10, "kris"));
-
- List hostTwoList = Arrays.asList(
- new HostWorker("192.168.1.1", 80, 20, "kris"),
- new HostWorker("192.168.1.2", 80, 10, "kris"),
- new HostWorker("192.168.1.3", 80, 10, "kris"));
-
- HostWorker result;
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.2", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.2", result.getIp());
-
- // add new host
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.3", result.getIp());
-
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.2", result.getIp());
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostTwoList);
- Assertions.assertEquals("192.168.1.3", result.getIp());
-
- // remove host3
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.2", result.getIp());
-
- result = selector.select(hostOneList);
- Assertions.assertEquals("192.168.1.1", result.getIp());
- }
-
- @Test
- public void testWeightRoundRobinSelector() {
- RoundRobinSelector selector = new RoundRobinSelector();
- HostWorker result;
- result = selector.select(
- Arrays.asList(new HostWorker("192.168.1.1", 11, 20, "kris"),
- new HostWorker("192.168.1.2", 22, 80, "kris")));
- Assertions.assertEquals("192.168.1.2", result.getIp());
- }
-
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManagerTest.java
deleted file mode 100644
index 38ece75e46..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterSlotManagerTest.java
+++ /dev/null
@@ -1,94 +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.server.master.registry;
-
-import org.apache.dolphinscheduler.common.enums.ServerStatus;
-import org.apache.dolphinscheduler.common.model.MasterHeartBeat;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.junit.jupiter.MockitoExtension;
-
-@ExtendWith(MockitoExtension.class)
-public class MasterSlotManagerTest {
-
- @InjectMocks
- private MasterSlotManager masterSlotManager = Mockito.spy(new MasterSlotManager());
-
- @Mock
- private MasterConfig masterConfig;
-
- @Test
- void testNormalMasterSlots() {
- // on normal Master side
- Mockito.when(masterConfig.getMasterAddress()).thenReturn("127.0.0.1:7777");
-
- sendHeartBeat(ServerStatus.BUSY, ServerStatus.NORMAL);
- Assertions.assertEquals(1, masterSlotManager.getMasterSize());
- Assertions.assertEquals(0, masterSlotManager.getSlot());
-
- sendHeartBeat(ServerStatus.NORMAL, ServerStatus.NORMAL);
- Assertions.assertEquals(2, masterSlotManager.getMasterSize());
- Assertions.assertEquals(1, masterSlotManager.getSlot());
- }
-
- @Test
- void testOverloadMasterSlots() {
- // on abnormal Master side
- Mockito.when(masterConfig.getMasterAddress()).thenReturn("127.0.0.1:6666");
-
- sendHeartBeat(ServerStatus.BUSY, ServerStatus.NORMAL);
- Assertions.assertEquals(0, masterSlotManager.getMasterSize());
- Assertions.assertEquals(0, masterSlotManager.getSlot());
-
- sendHeartBeat(ServerStatus.NORMAL, ServerStatus.NORMAL);
- Assertions.assertEquals(2, masterSlotManager.getMasterSize());
- Assertions.assertEquals(0, masterSlotManager.getSlot());
- }
-
- public void sendHeartBeat(ServerStatus serverStatus1, ServerStatus serverStatus2) {
- MasterSlotManager.SlotChangeListener slotChangeListener = masterSlotManager.new SlotChangeListener();
-
- Map masterNodeInfo = new HashMap<>();
- // generate heartbeat
- MasterHeartBeat masterHeartBeat1 = MasterHeartBeat.builder()
- .startupTime(System.currentTimeMillis())
- .serverStatus(serverStatus1)
- .host("127.0.0.1")
- .port(6666)
- .build();
- MasterHeartBeat masterHeartBeat2 = MasterHeartBeat.builder()
- .startupTime(System.currentTimeMillis())
- .serverStatus(serverStatus2)
- .host("127.0.0.1")
- .port(7777)
- .build();
- masterNodeInfo.put("127.0.0.1:6666", masterHeartBeat1);
- masterNodeInfo.put("127.0.0.1:7777", masterHeartBeat2);
-
- slotChangeListener.notify(masterNodeInfo);
- }
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java
deleted file mode 100644
index fd9e81549f..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java
+++ /dev/null
@@ -1,93 +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.server.master.registry;
-
-import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
-import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.registry.api.Event;
-import org.apache.dolphinscheduler.registry.api.RegistryClient;
-import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager;
-
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-public class ServerNodeManagerTest {
-
- @Mock
- RegistryClient registryClient;
-
- @Mock
- AlertDao alertDao;
-
- @Mock
- ListenerEventAlertManager listenerEventAlertManager;
-
- @InjectMocks
- ServerNodeManager serverNodeManager;
-
- @Test
- public void updateWorkerNodesTest() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
-
- MockitoAnnotations.initMocks(this);
- HashMap workerNodeMaps = new HashMap<>();
- workerNodeMaps.put("worker-node-1", JSONUtils.toJsonString(new WorkerHeartBeat()));
- workerNodeMaps.put("worker-node-2", JSONUtils.toJsonString(new WorkerHeartBeat()));
-
- Mockito.when(registryClient.getServerMaps(Mockito.any())).thenReturn(workerNodeMaps);
- Mockito.when(registryClient.isWorkerPath(Mockito.anyString())).thenReturn(true);
-
- // two worker server running (worker-node-1, worker-node-2)
- Method updateWorkerNodes = serverNodeManager.getClass().getDeclaredMethod("updateWorkerNodes");
- updateWorkerNodes.setAccessible(true);
- updateWorkerNodes.invoke(serverNodeManager);
-
- Map workerNodeInfo = serverNodeManager.getWorkerNodeInfo();
- Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-1"));
- Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2"));
-
- // receive remove event when worker-node-1 server stop
- ServerNodeManager.WorkerDataListener workerDataListener = serverNodeManager.new WorkerDataListener();
- Event event = new Event("", "/nodes/worker/worker-node-1", "", Event.Type.REMOVE);
- workerDataListener.notify(event);
-
- // check worker-node-1 not exist in cache
- workerNodeInfo = serverNodeManager.getWorkerNodeInfo();
- Assertions.assertFalse(workerNodeInfo.containsKey("worker-node-1"));
- Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2"));
-
- // worker-node-1 restart, getServerMaps(RegistryNodeType.WORKER) method return two worker
- updateWorkerNodes.invoke(serverNodeManager);
-
- // check cache
- workerNodeInfo = serverNodeManager.getWorkerNodeInfo();
- Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-1"));
- Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2"));
-
- }
-
-}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java
index f57c9b6a68..bbcc06c547 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java
@@ -38,8 +38,7 @@ public class MasterTaskDispatcherTest {
Mockito.when(masterConfig.getMasterAddress()).thenReturn("localhost:5678");
TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class);
- MasterTaskDispatcher masterTaskDispatcher =
- new MasterTaskDispatcher(taskEventService, masterConfig);
+ MasterTaskDispatcher masterTaskDispatcher = new MasterTaskDispatcher(masterConfig);
Host taskInstanceDispatchHost = masterTaskDispatcher.getTaskInstanceDispatchHost(taskExecuteRunnable)
.orElseThrow(() -> new IllegalArgumentException("Cannot get the "));
Assertions.assertEquals(masterConfig.getMasterAddress(), taskInstanceDispatchHost.getAddress());
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java
index 46c4f53e1a..88c0923e82 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java
@@ -19,10 +19,7 @@ package org.apache.dolphinscheduler.server.master.runner.dispatcher;
import org.apache.dolphinscheduler.extract.base.utils.Host;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
-import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager;
-import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
+import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer;
import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable;
import java.util.Optional;
@@ -37,13 +34,10 @@ import org.mockito.junit.jupiter.MockitoExtension;
public class WorkerTaskDispatcherTest {
@Test
- public void getTaskInstanceDispatchHost() throws WorkerGroupNotFoundException {
- TaskEventService taskEventService = Mockito.mock(TaskEventService.class);
- MasterConfig masterConfig = Mockito.mock(MasterConfig.class);
- HostManager hostManager = Mockito.mock(HostManager.class);
- Mockito.when(hostManager.select(Mockito.any())).thenReturn(Optional.of(Host.of("localhost:1234")));
- WorkerTaskDispatcher workerTaskDispatcher =
- new WorkerTaskDispatcher(taskEventService, masterConfig, hostManager);
+ public void getTaskInstanceDispatchHost() {
+ IWorkerLoadBalancer workerLoadBalancer = Mockito.mock(IWorkerLoadBalancer.class);
+ Mockito.when(workerLoadBalancer.select(Mockito.any())).thenReturn(Optional.of("localhost:1234"));
+ WorkerTaskDispatcher workerTaskDispatcher = new WorkerTaskDispatcher(workerLoadBalancer);
TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class);
Mockito.when(taskExecuteRunnable.getTaskExecutionContext()).thenReturn(new TaskExecutionContext());
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactoryTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactoryTest.java
new file mode 100644
index 0000000000..cf7b18d8af
--- /dev/null
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/MasterThreadFactoryTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.server.master.utils;
+
+import java.time.Duration;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.awaitility.Awaitility;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.truth.Truth;
+
+class MasterThreadFactoryTest {
+
+ @Test
+ void getDefaultSchedulerThreadExecutor() {
+ ScheduledExecutorService defaultSchedulerThreadExecutor =
+ MasterThreadFactory.getDefaultSchedulerThreadExecutor();
+ Truth.assertThat(defaultSchedulerThreadExecutor).isNotNull();
+
+ AtomicBoolean taskOneFlag = new AtomicBoolean(false);
+ defaultSchedulerThreadExecutor.scheduleWithFixedDelay(() -> {
+ taskOneFlag.set(true);
+ }, 0,
+ 1,
+ java.util.concurrent.TimeUnit.SECONDS);
+
+ AtomicBoolean taskTwoFlag = new AtomicBoolean(false);
+ defaultSchedulerThreadExecutor.scheduleWithFixedDelay(() -> {
+ taskTwoFlag.set(true);
+ }, 0,
+ 1,
+ java.util.concurrent.TimeUnit.SECONDS);
+
+ Awaitility.await()
+ .atMost(Duration.ofSeconds(5))
+ .untilAsserted(() -> {
+ Truth.assertThat(taskOneFlag.get()).isTrue();
+ Truth.assertThat(taskTwoFlag.get()).isTrue();
+ });
+
+ }
+}
diff --git a/dolphinscheduler-master/src/test/resources/application.yaml b/dolphinscheduler-master/src/test/resources/application.yaml
index 15f9199609..67950bfcf0 100644
--- a/dolphinscheduler-master/src/test/resources/application.yaml
+++ b/dolphinscheduler-master/src/test/resources/application.yaml
@@ -95,8 +95,6 @@ master:
exec-threads: 100
# master dispatch task number per batch, if all the tasks dispatch failed in a batch, will sleep 1s.
dispatch-task-number: 30
- # master host selector to select a suitable worker, default value: LowerWeight. Optional values include random, round_robin, lower_weight
- host-selector: lower_weight
# master heartbeat interval
max-heartbeat-interval: 10s
# master commit task retry times
@@ -115,6 +113,14 @@ master:
max-system-memory-usage-percentage-thresholds: 0.77
# Master max disk usage , when the master's disk usage is smaller then this value, master server can execute workflow.
max-disk-usage-percentage-thresholds: 0.77
+ worker-load-balancer-configuration-properties:
+ # RANDOM, ROUND_ROBIN, FIXED_WEIGHTED_ROUND_ROBIN, DYNAMIC_WEIGHTED_ROUND_ROBIN
+ type: DYNAMIC_WEIGHTED_ROUND_ROBIN
+ # dynamic-weight-config-properties only used in DYNAMIC_WEIGHTED_ROUND_ROBIN, the weight of memory-usage, cpu-usage, task-thread-pool-usage should sum to 100.
+ dynamic-weight-config-properties:
+ memory-usage-weight: 40
+ cpu-usage-weight: 30
+ task-thread-pool-usage-weight: 30
# failover interval, the unit is minute
failover-interval: 10m
# kill yarn / k8s application when failover taskInstance, default true
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java
index 2cfae44a47..53ae5f9449 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java
@@ -59,9 +59,15 @@ public class RegistryClient {
public RegistryClient(Registry registry) {
this.registry = registry;
- registry.put(RegistryNodeType.MASTER.getRegistryPath(), EMPTY, false);
- registry.put(RegistryNodeType.WORKER.getRegistryPath(), EMPTY, false);
- registry.put(RegistryNodeType.ALERT_SERVER.getRegistryPath(), EMPTY, false);
+ if (!registry.exists(RegistryNodeType.MASTER.getRegistryPath())) {
+ registry.put(RegistryNodeType.MASTER.getRegistryPath(), EMPTY, false);
+ }
+ if (!registry.exists(RegistryNodeType.WORKER.getRegistryPath())) {
+ registry.put(RegistryNodeType.WORKER.getRegistryPath(), EMPTY, false);
+ }
+ if (!registry.exists(RegistryNodeType.ALERT_SERVER.getRegistryPath())) {
+ registry.put(RegistryNodeType.ALERT_SERVER.getRegistryPath(), EMPTY, false);
+ }
}
public boolean isConnected() {
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueue.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueue.java
deleted file mode 100644
index 2d60eb2096..0000000000
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueue.java
+++ /dev/null
@@ -1,120 +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.service.queue;
-
-import org.apache.dolphinscheduler.common.model.Server;
-import org.apache.dolphinscheduler.common.utils.NetUtils;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-public class MasterPriorityQueue implements TaskPriorityQueue {
-
- /**
- * queue size
- */
- private static final Integer QUEUE_MAX_SIZE = 20;
-
- /**
- * queue
- */
- private PriorityBlockingQueue queue = new PriorityBlockingQueue<>(QUEUE_MAX_SIZE, new ServerComparator());
-
- private HashMap hostIndexMap = new HashMap<>();
-
- @Override
- public void put(Server serverInfo) {
- this.queue.put(serverInfo);
- refreshMasterList();
- }
-
- @Override
- public Server take() throws InterruptedException {
- return queue.take();
- }
-
- @Override
- public Server poll(long timeout, TimeUnit unit) {
- return queue.poll();
- }
-
- @Override
- public int size() {
- return queue.size();
- }
-
- public void putAll(Collection serverList) {
- for (Server server : serverList) {
- this.queue.put(server);
- }
- refreshMasterList();
- }
-
- public void remove(Server server) {
- this.queue.remove(server);
- }
-
- public void clear() {
- queue.clear();
- refreshMasterList();
- }
-
- private void refreshMasterList() {
- hostIndexMap.clear();
- int index = 0;
- for (Server server : getOrderedElements()) {
- String addr = NetUtils.getAddr(server.getHost(), server.getPort());
- hostIndexMap.put(addr, index);
- index += 1;
- }
- }
-
- /**
- * get ordered collection of priority queue
- *
- * @return ordered collection
- */
- Server[] getOrderedElements() {
- Server[] nQueue = queue.toArray(new Server[0]);
- Arrays.sort(nQueue, new ServerComparator());
- return nQueue;
- }
-
- public int getIndex(String addr) {
- if (!hostIndexMap.containsKey(addr)) {
- return -1;
- }
- return hostIndexMap.get(addr);
- }
-
- /**
- * server comparator, used to sort server by createTime in reverse order.
- */
- private class ServerComparator implements Comparator {
-
- @Override
- public int compare(Server o1, Server o2) {
- return o2.getCreateTime().compareTo(o1.getCreateTime());
- }
- }
-
-}
diff --git a/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueueTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueueTest.java
deleted file mode 100644
index a064c928e7..0000000000
--- a/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/MasterPriorityQueueTest.java
+++ /dev/null
@@ -1,101 +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.service.queue;
-
-import org.apache.dolphinscheduler.common.model.Server;
-
-import java.util.Arrays;
-import java.util.Date;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-public class MasterPriorityQueueTest {
-
- @Test
- public void getOrderedCollection() {
-
- MasterPriorityQueue queue = new MasterPriorityQueue();
-
- // Test empty queue
- Server[] emptyElements = queue.getOrderedElements();
- Assertions.assertArrayEquals(emptyElements, new Server[]{});
-
- // Test queue with fabricated servers
- queue.putAll(getServerList());
- Server[] orderElements = queue.getOrderedElements();
- Assertions.assertEquals(extractServerIds(orderElements), Arrays.asList(4, 2, 1, 3));
-
- }
-
- @Test
- public void refreshMasterList() {
- MasterPriorityQueue queue = new MasterPriorityQueue();
-
- // Test empty queue
- queue.clear();
- Assertions.assertEquals(queue.getIndex("127.0.0.1:124"), -1);
-
- // Test queue with fabricated servers
- queue.putAll(getServerList());
-
- Assertions.assertEquals(queue.getIndex("127.0.0.1:124"), 0);
- Assertions.assertEquals(queue.getIndex("127.0.0.1:122"), 1);
- Assertions.assertEquals(queue.getIndex("127.0.0.1:121"), 2);
- Assertions.assertEquals(queue.getIndex("127.0.0.1:123"), 3);
-
- }
-
- private List getServerList() {
-
- long baseTime = new Date().getTime();
-
- Server server1 = new Server();
- server1.setId(1);
- server1.setHost("127.0.0.1");
- server1.setPort(121);
- server1.setCreateTime(new Date(baseTime - 1000));
-
- Server server2 = new Server();
- server2.setId(2);
- server2.setHost("127.0.0.1");
- server2.setPort(122);
- server2.setCreateTime(new Date(baseTime + 1000));
-
- Server server3 = new Server();
- server3.setId(3);
- server3.setHost("127.0.0.1");
- server3.setPort(123);
- server3.setCreateTime(new Date(baseTime - 2000));
-
- Server server4 = new Server();
- server4.setId(4);
- server4.setHost("127.0.0.1");
- server4.setPort(124);
- server4.setCreateTime(new Date(baseTime + 2000));
-
- return Arrays.asList(server1, server2, server3, server4);
- }
-
- private List extractServerIds(Server[] servers) {
- return Arrays.stream(servers).map(Server::getId).collect(Collectors.toList());
- }
-
-}
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index 2da687d285..2a056305a7 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -165,10 +165,6 @@ master:
pre-exec-threads: 10
# master execute thread number to limit process instances in parallel
exec-threads: 10
- # master dispatch task number per batch
- dispatch-task-number: 3
- # master host selector to select a suitable worker, default value: LowerWeight. Optional values include random, round_robin, lower_weight
- host-selector: lower_weight
# master heartbeat interval
max-heartbeat-interval: 10s
# master commit task retry times
@@ -186,6 +182,14 @@ master:
max-system-memory-usage-percentage-thresholds: 0.9
# Master max disk usage , when the master's disk usage is smaller then this value, master server can execute workflow.
max-disk-usage-percentage-thresholds: 0.9
+ worker-load-balancer-configuration-properties:
+ # RANDOM, ROUND_ROBIN, FIXED_WEIGHTED_ROUND_ROBIN, DYNAMIC_WEIGHTED_ROUND_ROBIN
+ type: DYNAMIC_WEIGHTED_ROUND_ROBIN
+ # dynamic-weight-config-properties only used in DYNAMIC_WEIGHTED_ROUND_ROBIN, the weight of memory-usage, cpu-usage, task-thread-pool-usage should sum to 100.
+ dynamic-weight-config-properties:
+ memory-usage-weight: 30
+ cpu-usage-weight: 30
+ task-thread-pool-usage-weight: 40
# failover interval
failover-interval: 10m
# kill yarn/k8s application when failover taskInstance, default true
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java
index 99645f89a4..5a1fc5ab9b 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java
@@ -80,6 +80,10 @@ public class WorkerTaskExecutorThreadPool {
return Math.min(WorkerTaskExecutorHolder.size(), workerConfig.getExecThreads());
}
+ public double getThreadPoolUsage() {
+ return (double) WorkerTaskExecutorHolder.size() / workerConfig.getExecThreads();
+ }
+
/**
* Kill tasks that have not been executed, e.g. waiting in the queue
*/
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
index 62bee33a90..13f585d92a 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
@@ -73,8 +73,7 @@ public class WorkerHeartBeatTask extends BaseHeartBeatTask {
.diskUsage(systemMetrics.getDiskUsedPercentage())
.processId(processId)
.workerHostWeight(workerConfig.getHostWeight())
- .threadPoolUsage(workerTaskExecutorThreadPool.getRunningTaskExecutorSize()
- + workerTaskExecutorThreadPool.getWaitingTaskExecutorSize())
+ .threadPoolUsage(workerTaskExecutorThreadPool.getThreadPoolUsage())
.serverStatus(serverStatus)
.host(NetUtils.getHost())
.port(workerConfig.getListenPort())