From 00b8dc69c3a9e0517e45f2bd3f9f1c47100d30ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Mon, 15 Dec 2025 19:52:24 +0800 Subject: [PATCH 01/18] Add dispatch timeout checking logic --- .../server/master/config/MasterConfig.java | 6 ++ .../PhysicalTaskExecutorClientDelegator.java | 22 ++++- .../dispatcher/WorkerGroupDispatcher.java | 81 +++++++++++++++++-- .../WorkerGroupDispatcherCoordinator.java | 9 ++- .../WorkerGroupNotFoundException.java | 4 +- .../dispatch/WorkerNotFoundException.java | 25 ++++++ .../server/master/utils/ExceptionUtils.java | 10 +++ .../src/main/resources/application.yaml | 3 + .../dispatcher/WorkerGroupDispatcherTest.java | 4 +- .../plugin/task/api/TaskExecutionContext.java | 5 ++ 10 files changed, 152 insertions(+), 17 deletions(-) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java 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 220062bd5799..da99628bdb6e 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 @@ -74,6 +74,12 @@ public class MasterConfig implements Validator { */ private String masterRegistryPath; + /** + * Maximum time allowed for a task to be successfully dispatched. + * Default: 5 minutes. + */ + private Duration dispatchTimeout = Duration.ofMinutes(5); + @Override public boolean supports(Class clazz) { return MasterConfig.class.isAssignableFrom(clazz); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index f7566d3133b1..9de51915fc79 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -24,10 +24,13 @@ import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cluster.ClusterManager; import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; @@ -55,18 +58,29 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD @Autowired private IWorkerLoadBalancer workerLoadBalancer; + @Autowired + private ClusterManager clusterManager; + @Override public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); final String taskName = taskExecutionContext.getTaskName(); + final String workerGroup = taskExecutionContext.getWorkerGroup(); + if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { + throw new WorkerGroupNotFoundException( + String.format("Cannot find worker group to dispatch Task[id=%s, name=%s, workerGroup=%s]", + taskExecutionContext.getTaskInstanceId(), taskName, + workerGroup)); + } final String physicalTaskExecutorAddress = workerLoadBalancer - .select(taskExecutionContext.getWorkerGroup()) + .select(workerGroup) .map(Host::of) .map(Host::getAddress) - .orElseThrow(() -> new TaskDispatchException( - String.format("Cannot find the host to dispatch Task[id=%s, name=%s, workerGroup=%s]", + .orElseThrow(() -> new WorkerNotFoundException( + String.format( + "Cannot find available worker host to dispatch Task[id=%s, name=%s, workerGroup=%s]", taskExecutionContext.getTaskInstanceId(), taskName, - taskExecutionContext.getWorkerGroup()))); + workerGroup))); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 28834e27e7e1..f5d8a14b09f4 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -18,12 +18,19 @@ package org.apache.dolphinscheduler.server.master.engine.task.dispatcher; import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; +import java.time.Duration; +import java.util.Date; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,11 +55,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread { private final AtomicBoolean runningFlag = new AtomicBoolean(false); - public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient) { + private final Duration dispatchTimeout; + + public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient, + Duration dispatchTimeout) { super("WorkerGroupTaskDispatcher-" + workerGroupName); this.taskExecutorClient = taskExecutorClient; this.workerGroupEventBus = new TaskDispatchableEventBus<>(); this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet(); + this.dispatchTimeout = dispatchTimeout; log.info("Initialize WorkerGroupDispatcher: {}", this.getName()); } @@ -84,23 +95,77 @@ public void run() { } private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { + final int taskId = taskExecutionRunnable.getId(); + final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); + final long timeoutMs = this.dispatchTimeout.toMillis(); try { - if (!waitingDispatchTaskIds.remove(taskExecutionRunnable.getId())) { + if (!waitingDispatchTaskIds.remove(taskId)) { log.info( "The task: {} doesn't exist in waitingDispatchTaskIds(it might be paused or killed), will skip dispatch", - taskExecutionRunnable.getId()); + taskId); return; } taskExecutorClient.dispatch(taskExecutionRunnable); - } catch (Exception e) { + } catch (TaskDispatchException ex) { + // Checks whether the given task has exceeded its allowed dispatch timeout. + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + if (elapsed > timeoutMs) { + handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); + return; + } + // If dispatch failed, will put the task back to the queue // The task will be dispatched after waiting time. // the waiting time will increase multiple of times, but will not exceed 60 seconds - long waitingTimeMills = Math.min( + long waitingTimeMillis = Math.min( taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L); - dispatchTask(taskExecutionRunnable, waitingTimeMills); - log.error("Dispatch Task: {} failed will retry after: {}/ms", taskExecutionRunnable.getId(), - waitingTimeMills, e); + dispatchTask(taskExecutionRunnable, waitingTimeMillis); + log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskId, + waitingTimeMillis, ex); + } + } + + /** + * Marks the specified task as fatally failed due to an unrecoverable dispatch error,such as timeout or persistent client failure. + * Once this method is called, the task is considered permanently failed and will not be retried. + * + * @param taskExecutionRunnable the task to mark as fatally failed; must not be null + * @param exception the dispatch exception that triggered this failure handling; must not be null + * @param elapsed the time (in milliseconds) already spent attempting to dispatch the task + * @param timeoutMs the configured dispatch timeout threshold (in milliseconds) + */ + private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception, + long elapsed, long timeoutMs) { + int taskId = taskExecutionRunnable.getId(); + int workflowId = taskExecutionRunnable.getWorkflowInstance().getId(); + + log.warn("[DISPATCH_FAILED] taskId: {}, workflowId: {}, timed out after {} ms (limit: {} ms))", taskId, + workflowId, elapsed, timeoutMs); + + if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) { + log.error("[DISPATCH_FAILED] Worker group not found. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); + } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { + log.error("[DISPATCH_FAILED] No available workers. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); + } else { + log.error("[DISPATCH_FAILED] Unexpected dispatch error. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java index a85674c6f4d7..b3fea52976de 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.engine.task.dispatcher; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; @@ -41,8 +42,11 @@ public class WorkerGroupDispatcherCoordinator implements AutoCloseable { private final ConcurrentHashMap workerGroupDispatcherMap; - public WorkerGroupDispatcherCoordinator() { + private final MasterConfig masterConfig; + + public WorkerGroupDispatcherCoordinator(final MasterConfig masterConfig) { workerGroupDispatcherMap = new ConcurrentHashMap<>(); + this.masterConfig = masterConfig; } public void start() { @@ -99,7 +103,8 @@ public void close() throws Exception { private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) { return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> { - WorkerGroupDispatcher workerGroupDispatcher = new WorkerGroupDispatcher(wg, taskExecutorClient); + WorkerGroupDispatcher workerGroupDispatcher = + new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeout()); workerGroupDispatcher.start(); return workerGroupDispatcher; }); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java index c9d130172589..ddefa6fe0a04 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java @@ -19,7 +19,7 @@ public class WorkerGroupNotFoundException extends TaskDispatchException { - public WorkerGroupNotFoundException(String workerGroup) { - super("Cannot find worker group: " + workerGroup); + public WorkerGroupNotFoundException(String message) { + super(message); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java new file mode 100644 index 000000000000..10860ace2640 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java @@ -0,0 +1,25 @@ +/* + * 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.exception.dispatch; + +public class WorkerNotFoundException extends TaskDispatchException { + + public WorkerNotFoundException(String message) { + super(message); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index 9103bc5075d8..0997007057e6 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -17,6 +17,9 @@ package org.apache.dolphinscheduler.server.master.utils; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; + import org.springframework.dao.DataAccessResourceFailureException; public class ExceptionUtils { @@ -25,4 +28,11 @@ public static boolean isDatabaseConnectedFailedException(Throwable e) { return e instanceof DataAccessResourceFailureException; } + public static boolean isWorkerGroupNotFoundException(Throwable e) { + return e instanceof WorkerGroupNotFoundException; + } + + public static boolean isWorkerNotFoundException(Throwable e) { + return e instanceof WorkerNotFoundException; + } } diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 39a0f4311ab7..4980597de057 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -111,6 +111,9 @@ master: # Master max concurrent workflow instances, when the master's workflow instance count exceeds this value, master server will be marked as busy. max-concurrent-workflow-instances: 2147483647 worker-group-refresh-interval: 5m + # Timeout for task dispatching. If a task cannot be successfully dispatched to any worker within this duration, + # it will be marked as failed. Prevents tasks from being stuck indefinitely in the dispatch queue. + dispatch-timeout: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index 77525cb1810e..a7f4d3dba058 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; @@ -46,7 +47,8 @@ class WorkerGroupDispatcherTest { @BeforeEach void setUp() { taskExecutorClient = mock(ITaskExecutorClient.class); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient); + final MasterConfig masterConfig = new MasterConfig(); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeout()); } @Test diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java index a8f46435c1bb..fea0b50d32eb 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java @@ -126,6 +126,11 @@ public class TaskExecutionContext implements Serializable { private boolean failover; + /** + * Timestamp (ms) when the task was first enqueued for dispatch. + */ + private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis(); + public int increaseDispatchFailTimes() { return ++dispatchFailTimes; } From bb089f3c6e7c0a0c6bcbfe94d8761d3711cba425 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 11:52:20 +0800 Subject: [PATCH 02/18] add dispatch-timeout-checker switch --- .../server/master/config/MasterConfig.java | 20 +++++++-- .../MasterDispatchTimeoutCheckerConfig.java | 42 +++++++++++++++++++ .../dispatcher/WorkerGroupDispatcher.java | 22 +++++----- .../WorkerGroupDispatcherCoordinator.java | 2 +- .../src/main/resources/application.yaml | 8 ++-- .../dispatcher/WorkerGroupDispatcherTest.java | 3 +- 6 files changed, 79 insertions(+), 18 deletions(-) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java 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 da99628bdb6e..ee8762c68378 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 @@ -75,10 +75,11 @@ public class MasterConfig implements Validator { private String masterRegistryPath; /** - * Maximum time allowed for a task to be successfully dispatched. - * Default: 5 minutes. + * Configuration for the master's task dispatch timeout check mechanism. + * This controls whether the system enforces a time limit for dispatching tasks to workers, + * and if so, how long to wait before marking a task as failed due to dispatch timeout. */ - private Duration dispatchTimeout = Duration.ofMinutes(5); + private MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker = new MasterDispatchTimeoutCheckerConfig(); @Override public boolean supports(Class clazz) { @@ -103,6 +104,19 @@ public void validate(Object target, Errors errors) { if (masterConfig.getWorkerGroupRefreshInterval().getSeconds() < 10) { errors.rejectValue("worker-group-refresh-interval", null, "should >= 10s"); } + + // Validate dispatch timeout checker config + MasterDispatchTimeoutCheckerConfig timeoutChecker = masterConfig.getDispatchTimeoutChecker(); + if (timeoutChecker != null && timeoutChecker.isEnabled()) { + if (timeoutChecker.getTimeoutDuration() == null) { + errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + "must be specified when dispatch timeout checker is enabled"); + } else if (timeoutChecker.getTimeoutDuration().toMillis() <= 0) { + errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + "must be a positive duration (e.g., '2m', '5m', '30m')"); + } + } + if (StringUtils.isEmpty(masterConfig.getMasterAddress())) { masterConfig.setMasterAddress(NetUtils.getAddr(masterConfig.getListenPort())); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java new file mode 100644 index 000000000000..aa83d1eba365 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java @@ -0,0 +1,42 @@ +/* + * 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.config; + +import java.time.Duration; + +import lombok.Data; + +/** + * Configuration for the master's task dispatch timeout checker. + * If enabled, tasks that remain in the dispatch queue longer than {@link #timeoutDuration} will be marked as failed to prevent indefinite queuing. + */ +@Data +public class MasterDispatchTimeoutCheckerConfig { + + /** + * Whether to enable the dispatch timeout checking mechanism. + */ + private boolean enabled = false; + + /** + * Maximum allowed time for a task to be dispatched to a worker. + * Tasks exceeding this duration in the dispatch queue will be failed. + * Examples: "2m", "5m", "30m". Defaults to 5 minutes. + */ + private Duration timeoutDuration = Duration.ofMinutes(5); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index f5d8a14b09f4..2fb5dd4d35d2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; @@ -29,7 +30,6 @@ import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; -import java.time.Duration; import java.util.Date; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -55,15 +55,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread { private final AtomicBoolean runningFlag = new AtomicBoolean(false); - private final Duration dispatchTimeout; + private final MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker; public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient, - Duration dispatchTimeout) { + MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker) { super("WorkerGroupTaskDispatcher-" + workerGroupName); this.taskExecutorClient = taskExecutorClient; this.workerGroupEventBus = new TaskDispatchableEventBus<>(); this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet(); - this.dispatchTimeout = dispatchTimeout; + this.dispatchTimeoutChecker = dispatchTimeoutChecker; log.info("Initialize WorkerGroupDispatcher: {}", this.getName()); } @@ -97,7 +97,6 @@ public void run() { private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { final int taskId = taskExecutionRunnable.getId(); final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); - final long timeoutMs = this.dispatchTimeout.toMillis(); try { if (!waitingDispatchTaskIds.remove(taskId)) { log.info( @@ -107,11 +106,14 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { } taskExecutorClient.dispatch(taskExecutionRunnable); } catch (TaskDispatchException ex) { - // Checks whether the given task has exceeded its allowed dispatch timeout. - long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); - if (elapsed > timeoutMs) { - handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); - return; + if (dispatchTimeoutChecker.isEnabled()) { + // Checks whether the given task has exceeded its allowed dispatch timeout. + long timeoutMs = this.dispatchTimeoutChecker.getTimeoutDuration().toMillis(); + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + if (elapsed > timeoutMs) { + handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); + return; + } } // If dispatch failed, will put the task back to the queue diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java index b3fea52976de..66663a923a1e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java @@ -104,7 +104,7 @@ public void close() throws Exception { private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) { return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> { WorkerGroupDispatcher workerGroupDispatcher = - new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeout()); + new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); workerGroupDispatcher.start(); return workerGroupDispatcher; }); diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 4980597de057..d3a62eced67e 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -111,9 +111,11 @@ master: # Master max concurrent workflow instances, when the master's workflow instance count exceeds this value, master server will be marked as busy. max-concurrent-workflow-instances: 2147483647 worker-group-refresh-interval: 5m - # Timeout for task dispatching. If a task cannot be successfully dispatched to any worker within this duration, - # it will be marked as failed. Prevents tasks from being stuck indefinitely in the dispatch queue. - dispatch-timeout: 5m + # Task dispatch timeout check (currently disabled). + # When enabled, tasks not dispatched within this duration are marked as failed. + dispatch-timeout-checker: + enabled: false + timeout-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index a7f4d3dba058..a60d87e50d6a 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -48,7 +48,8 @@ class WorkerGroupDispatcherTest { void setUp() { taskExecutorClient = mock(ITaskExecutorClient.class); final MasterConfig masterConfig = new MasterConfig(); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeout()); + dispatcher = + new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); } @Test From fbd7edf30e7230bdf1604a4135c5aa33e98dd4bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 16:19:10 +0800 Subject: [PATCH 03/18] add dispatch-timeout-checker switch --- .../server/master/config/MasterConfig.java | 9 ++++--- .../MasterDispatchTimeoutCheckerConfig.java | 4 +-- .../PhysicalTaskExecutorClientDelegator.java | 13 +++------ .../dispatcher/WorkerGroupDispatcher.java | 27 ++++++++----------- ...n.java => NoAvailableWorkerException.java} | 6 ++--- .../WorkerGroupNotFoundException.java | 4 +-- .../server/master/utils/ExceptionUtils.java | 1 - .../src/main/resources/application.yaml | 2 +- .../src/test/resources/application.yaml | 5 ++++ .../plugin/task/api/TaskExecutionContext.java | 4 +-- 10 files changed, 34 insertions(+), 41 deletions(-) rename dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/{WorkerNotFoundException.java => NoAvailableWorkerException.java} (80%) 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 ee8762c68378..b727ad508b79 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 @@ -108,11 +108,11 @@ public void validate(Object target, Errors errors) { // Validate dispatch timeout checker config MasterDispatchTimeoutCheckerConfig timeoutChecker = masterConfig.getDispatchTimeoutChecker(); if (timeoutChecker != null && timeoutChecker.isEnabled()) { - if (timeoutChecker.getTimeoutDuration() == null) { - errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + if (timeoutChecker.getMaxTaskDispatchDuration() == null) { + errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be specified when dispatch timeout checker is enabled"); - } else if (timeoutChecker.getTimeoutDuration().toMillis() <= 0) { - errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + } else if (timeoutChecker.getMaxTaskDispatchDuration().toMillis() <= 0) { + errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be a positive duration (e.g., '2m', '5m', '30m')"); } } @@ -142,6 +142,7 @@ private void printConfig() { "\n command-fetch-strategy: " + commandFetchStrategy + "\n worker-load-balancer-configuration-properties: " + workerLoadBalancerConfigurationProperties + + "\n dispatchTimeoutChecker: " + dispatchTimeoutChecker + "\n****************************Master Configuration**************************************"; log.info(config); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java index aa83d1eba365..0fc8467f47ce 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java @@ -23,7 +23,7 @@ /** * Configuration for the master's task dispatch timeout checker. - * If enabled, tasks that remain in the dispatch queue longer than {@link #timeoutDuration} will be marked as failed to prevent indefinite queuing. + * If enabled, tasks that remain in the dispatch queue longer than {@link #maxTaskDispatchDuration} will be marked as failed to prevent indefinite queuing. */ @Data public class MasterDispatchTimeoutCheckerConfig { @@ -38,5 +38,5 @@ public class MasterDispatchTimeoutCheckerConfig { * Tasks exceeding this duration in the dispatch queue will be failed. * Examples: "2m", "5m", "30m". Defaults to 5 minutes. */ - private Duration timeoutDuration = Duration.ofMinutes(5); + private Duration maxTaskDispatchDuration = Duration.ofMinutes(5); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 9de51915fc79..271c03f70a34 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -28,9 +28,9 @@ import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; -import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; @@ -67,20 +67,13 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws final String taskName = taskExecutionContext.getTaskName(); final String workerGroup = taskExecutionContext.getWorkerGroup(); if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { - throw new WorkerGroupNotFoundException( - String.format("Cannot find worker group to dispatch Task[id=%s, name=%s, workerGroup=%s]", - taskExecutionContext.getTaskInstanceId(), taskName, - workerGroup)); + throw new WorkerGroupNotFoundException(workerGroup); } final String physicalTaskExecutorAddress = workerLoadBalancer .select(workerGroup) .map(Host::of) .map(Host::getAddress) - .orElseThrow(() -> new WorkerNotFoundException( - String.format( - "Cannot find available worker host to dispatch Task[id=%s, name=%s, workerGroup=%s]", - taskExecutionContext.getTaskInstanceId(), taskName, - workerGroup))); + .orElseThrow(() -> new NoAvailableWorkerException(workerGroup)); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 2fb5dd4d35d2..0d8f03ad3008 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -95,21 +95,21 @@ public void run() { } private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { - final int taskId = taskExecutionRunnable.getId(); + final int taskInstanceId = taskExecutionRunnable.getId(); final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); try { - if (!waitingDispatchTaskIds.remove(taskId)) { + if (!waitingDispatchTaskIds.remove(taskInstanceId)) { log.info( "The task: {} doesn't exist in waitingDispatchTaskIds(it might be paused or killed), will skip dispatch", - taskId); + taskInstanceId); return; } taskExecutorClient.dispatch(taskExecutionRunnable); } catch (TaskDispatchException ex) { if (dispatchTimeoutChecker.isEnabled()) { // Checks whether the given task has exceeded its allowed dispatch timeout. - long timeoutMs = this.dispatchTimeoutChecker.getTimeoutDuration().toMillis(); - long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + long timeoutMs = this.dispatchTimeoutChecker.getMaxTaskDispatchDuration().toMillis(); + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchTime(); if (elapsed > timeoutMs) { handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); return; @@ -122,7 +122,7 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { long waitingTimeMillis = Math.min( taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L); dispatchTask(taskExecutionRunnable, waitingTimeMillis); - log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskId, + log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskInstanceId, waitingTimeMillis, ex); } } @@ -138,31 +138,26 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { */ private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception, long elapsed, long timeoutMs) { - int taskId = taskExecutionRunnable.getId(); - int workflowId = taskExecutionRunnable.getWorkflowInstance().getId(); + final String taskName = taskExecutionRunnable.getName(); - log.warn("[DISPATCH_FAILED] taskId: {}, workflowId: {}, timed out after {} ms (limit: {} ms))", taskId, - workflowId, elapsed, timeoutMs); + log.warn("[DISPATCH_FAILED] taskName: {}, timed out after {} ms (limit: {} ms))", taskName, elapsed, timeoutMs); if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) { - log.error("[DISPATCH_FAILED] Worker group not found. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, Worker group not found.", taskName, exception); final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { - log.error("[DISPATCH_FAILED] No available workers. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); } else { - log.error("[DISPATCH_FAILED] Unexpected dispatch error. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, Unexpected dispatch error.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java similarity index 80% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java index 10860ace2640..d51e2342cc2b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java @@ -17,9 +17,9 @@ package org.apache.dolphinscheduler.server.master.exception.dispatch; -public class WorkerNotFoundException extends TaskDispatchException { +public class NoAvailableWorkerException extends TaskDispatchException { - public WorkerNotFoundException(String message) { - super(message); + public NoAvailableWorkerException(String workerGroup) { + super("Cannot find available worker under worker group: " + workerGroup); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java index ddefa6fe0a04..c9d130172589 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java @@ -19,7 +19,7 @@ public class WorkerGroupNotFoundException extends TaskDispatchException { - public WorkerGroupNotFoundException(String message) { - super(message); + public WorkerGroupNotFoundException(String workerGroup) { + super("Cannot find worker group: " + workerGroup); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index 0997007057e6..fe70a69ec02c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -18,7 +18,6 @@ package org.apache.dolphinscheduler.server.master.utils; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; -import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.springframework.dao.DataAccessResourceFailureException; diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index d3a62eced67e..1f741559efb5 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -115,7 +115,7 @@ master: # When enabled, tasks not dispatched within this duration are marked as failed. dispatch-timeout-checker: enabled: false - timeout-duration: 5m + max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/resources/application.yaml b/dolphinscheduler-master/src/test/resources/application.yaml index 3aca1e3142eb..548a1a84291c 100644 --- a/dolphinscheduler-master/src/test/resources/application.yaml +++ b/dolphinscheduler-master/src/test/resources/application.yaml @@ -73,6 +73,11 @@ master: cpu-usage-weight: 30 task-thread-pool-usage-weight: 30 worker-group-refresh-interval: 5m + # Task dispatch timeout check (currently disabled). + # When enabled, tasks not dispatched within this duration are marked as failed. + dispatch-timeout-checker: + enabled: false + max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java index fea0b50d32eb..e0fc55922640 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java @@ -127,9 +127,9 @@ public class TaskExecutionContext implements Serializable { private boolean failover; /** - * Timestamp (ms) when the task was first enqueued for dispatch. + * Timestamp (ms) when the task was first dispatched. */ - private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis(); + private final long firstDispatchTime = System.currentTimeMillis(); public int increaseDispatchFailTimes() { return ++dispatchFailTimes; From ab84035ca47949f0d18b8db23929fb206486e642 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 16:45:39 +0800 Subject: [PATCH 04/18] update ExceptionUtils --- .../master/engine/task/dispatcher/WorkerGroupDispatcher.java | 2 +- .../dolphinscheduler/server/master/utils/ExceptionUtils.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 0d8f03ad3008..12e9acb457c9 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -149,7 +149,7 @@ private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); - } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { + } else if (ExceptionUtils.isNoAvailableWorkerException(exception)) { log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index fe70a69ec02c..892dfab065e0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.utils; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; import org.springframework.dao.DataAccessResourceFailureException; @@ -31,7 +32,7 @@ public static boolean isWorkerGroupNotFoundException(Throwable e) { return e instanceof WorkerGroupNotFoundException; } - public static boolean isWorkerNotFoundException(Throwable e) { - return e instanceof WorkerNotFoundException; + public static boolean isNoAvailableWorkerException(Throwable e) { + return e instanceof NoAvailableWorkerException; } } From f193af6ba2193a694177be506a2f2f0ea62ce1cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 19:53:38 +0800 Subject: [PATCH 05/18] add test for dispatch timeout checker --- .../dispatcher/WorkerGroupDispatcherTest.java | 296 ++++++++++++++++++ 1 file changed, 296 insertions(+) diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index a60d87e50d6a..fb63676867fd 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -19,22 +19,37 @@ import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; +import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; import java.time.Duration; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.InOrder; @@ -141,4 +156,285 @@ void dispatch_TaskDispatchFails_RetryLogicWorks() throws TaskDispatchException { .untilAsserted(() -> verify(taskExecutorClient, times(2)).dispatch(taskExecutionRunnable)); } + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + WorkerGroupNotFoundException ex = new WorkerGroupNotFoundException("no worker group"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 500); + + WorkerGroupNotFoundException ex = new WorkerGroupNotFoundException("worker group not found"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFatalLifecycleEvent && + ((TaskFatalLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Dispatcher configured with a 5-minute timeout (enabled) + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + // Mock task with first dispatch time set to 100ms ago → well within timeout window + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect actual dispatch invocation + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub client to throw WorkerGroupNotFoundException and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Confirm dispatch was attempted + throw new WorkerGroupNotFoundException("Worker group 'TestGroup' does not exist"); + }).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + // When: Start dispatcher and dispatch the task + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Wait up to 1 second for the dispatch attempt to complete + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published because timeout has NOT been exceeded + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + NoAvailableWorkerException ex = new NoAvailableWorkerException("no worker"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given: enable timeout (200ms), task already waited 500ms + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = + mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis() - 500); + + NoAvailableWorkerException ex = new NoAvailableWorkerException("no worker"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && + ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Configure dispatcher with a 5-minute dispatch timeout (enabled) + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + // Mock task with first dispatch time set to 100ms ago → ensures it's NOT timed out yet + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect when dispatch is actually invoked (avoids timing flakiness) + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub the client to throw NoAvailableWorkerException on dispatch and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Signal that dispatch was attempted + throw new NoAvailableWorkerException("no worker"); + }).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + // When: Start dispatcher and trigger task dispatch + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Wait up to 1 second for the dispatch attempt to occur (ensures async execution completes) + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published since timeout has NOT been exceeded + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + TaskDispatchException ex = new TaskDispatchException("generic dispatch error"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 500); + + TaskDispatchException ex = new TaskDispatchException("generic dispatch error"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && + ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Dispatcher configured with a 5-minute dispatch timeout (enabled) + MasterDispatchTimeoutCheckerConfig config = new MasterDispatchTimeoutCheckerConfig(); + config.setEnabled(true); + config.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, config); + + // Mock task with first dispatch time set to 100ms ago → well within timeout window + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect when dispatch is actually invoked + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub client to throw a generic TaskDispatchException and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Confirm dispatch attempt occurred + throw new TaskDispatchException("Generic dispatch error"); + }).when(taskExecutorClient).dispatch(task); + + // When: Start dispatcher and trigger task dispatch + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // Wait up to 1 second for the dispatch attempt to complete (handles async execution) + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published because timeout has NOT been exceeded + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + private ITaskExecutionRunnable mockTaskExecutionRunnableWithFirstDispatchTime(long firstDispatchTime) { + ITaskExecutionRunnable taskExecutionRunnable = mock(ITaskExecutionRunnable.class); + TaskInstance taskInstance = mock(TaskInstance.class); + WorkflowInstance workflowInstance = mock(WorkflowInstance.class); + WorkflowEventBus eventBus = mock(WorkflowEventBus.class); + + TaskExecutionContext context = mock(TaskExecutionContext.class); + when(context.getFirstDispatchTime()).thenReturn(firstDispatchTime); + + when(taskExecutionRunnable.getTaskInstance()).thenReturn(taskInstance); + when(taskExecutionRunnable.getWorkflowInstance()).thenReturn(workflowInstance); + when(taskExecutionRunnable.getWorkflowEventBus()).thenReturn(eventBus); + when(taskExecutionRunnable.getId()).thenReturn(ThreadLocalRandom.current().nextInt(1000, 9999)); + when(taskExecutionRunnable.getTaskExecutionContext()).thenReturn(context); + + return taskExecutionRunnable; + } } From 584ad2d8b3fdc82cd16add505f4027aab18bfd3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Fri, 19 Dec 2025 14:50:16 +0800 Subject: [PATCH 06/18] add TaskFatalLifecycleEvent and handler --- .../PhysicalTaskExecutorClientDelegator.java | 4 +++ .../lifecycle/TaskLifecycleEventType.java | 4 +++ .../statemachine/AbstractTaskStateAction.java | 33 +++++++++++++++++++ .../task/statemachine/ITaskStateAction.java | 9 +++++ 4 files changed, 50 insertions(+) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 271c03f70a34..96331b2dcae9 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -66,9 +66,13 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); final String taskName = taskExecutionContext.getTaskName(); final String workerGroup = taskExecutionContext.getWorkerGroup(); + + // workerGroup not exist if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { throw new WorkerGroupNotFoundException(workerGroup); } + + // select an available worker from the worker group; throws NoAvailableWorkerException if none is available. final String physicalTaskExecutorAddress = workerLoadBalancer .select(workerGroup) .map(Host::of) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java index 5ddcf13c8917..fb12ccb60375 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java @@ -29,6 +29,10 @@ public enum TaskLifecycleEventType implements ILifecycleEventType { * Dispatch the task instance to target. */ DISPATCH, + /** + * Task instance encounters catastrophic failure(such as initialization failure), it will enter a failed state. + */ + FATAL, /** * The task instance is dispatched to the target executor server. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java index ad0e652d4866..738f284b9a7b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java @@ -36,6 +36,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; @@ -99,6 +100,38 @@ protected void releaseTaskInstanceResourcesIfNeeded(final ITaskExecutionRunnable } } + @Override + public void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + releaseTaskInstanceResourcesIfNeeded(taskExecutionRunnable); + persistentTaskInstanceFatalEventToDB(taskExecutionRunnable, taskFatalEvent); + + if (taskExecutionRunnable.isTaskInstanceCanRetry()) { + taskExecutionRunnable.getWorkflowEventBus().publish(TaskRetryLifecycleEvent.of(taskExecutionRunnable)); + return; + } + + // If all successors are condition tasks, then the task will not be marked as failure. + // And the DAG will continue to execute. + final IWorkflowExecutionGraph workflowExecutionGraph = taskExecutionRunnable.getWorkflowExecutionGraph(); + if (workflowExecutionGraph.isAllSuccessorsAreConditionTask(taskExecutionRunnable)) { + mergeTaskVarPoolToWorkflow(workflowExecutionRunnable, taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + return; + } + taskExecutionRunnable.getWorkflowExecutionGraph().markTaskExecutionRunnableChainFailure(taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + } + + private void persistentTaskInstanceFatalEventToDB(final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + taskInstance.setState(TaskExecutionStatus.FAILURE); + taskInstance.setEndTime(taskFatalEvent.getEndTime()); + taskInstanceDao.updateById(taskInstance); + } + @Override public void onDispatchedEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java index a041de5e3ca6..f60c3ae13532 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailoverLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; @@ -91,6 +92,14 @@ void onDispatchEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskDispatchLifecycleEvent taskDispatchEvent); + /** + * Perform the necessary actions when the task in a certain state receive a {@link TaskFatalLifecycleEvent}. + *

This method is called when the task encounters catastrophic failure (e.g., initialization failure). + */ + void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent); + /** * Perform the necessary actions when the task in a certain state receive a {@link TaskDispatchedLifecycleEvent}. *

This method is called when the task has been dispatched to executor. From 877db0634c794a7de4016b151016bd58b04f1217 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Fri, 19 Dec 2025 14:50:53 +0800 Subject: [PATCH 07/18] add TaskFatalLifecycleEvent and handler --- .../event/TaskFatalLifecycleEvent.java | 52 +++++++++++++++++++ .../TaskFatalLifecycleEventHandler.java | 44 ++++++++++++++++ 2 files changed, 96 insertions(+) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java new file mode 100644 index 000000000000..88cfc9a478f6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java @@ -0,0 +1,52 @@ +/* + * 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.engine.task.lifecycle.event; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Date; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +@AllArgsConstructor +public class TaskFatalLifecycleEvent extends AbstractTaskLifecycleEvent { + + private final ITaskExecutionRunnable taskExecutionRunnable; + + private final Date endTime; + + @Override + public ILifecycleEventType getEventType() { + return TaskLifecycleEventType.FATAL; + } + + @Override + public String toString() { + return "TaskFatalLifecycleEvent{" + + "task=" + taskExecutionRunnable.getName() + + ", endTime=" + endTime + + '}'; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java new file mode 100644 index 000000000000..8277d8b2caa7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java @@ -0,0 +1,44 @@ +/* + * 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.engine.task.lifecycle.handler; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; +import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; + +import org.springframework.stereotype.Component; + +@Component +public class TaskFatalLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + + @Override + public void handle(final ITaskStateAction taskStateAction, + final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + taskStateAction.onFatalEvent(workflowExecutionRunnable, taskExecutionRunnable, taskFatalEvent); + } + + @Override + public ILifecycleEventType matchEventType() { + return TaskLifecycleEventType.FATAL; + } +} From cb44db7c38d6a853603d8f8348560a80be531c4f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Mon, 15 Dec 2025 19:52:24 +0800 Subject: [PATCH 08/18] Add dispatch timeout checking logic --- .../server/master/config/MasterConfig.java | 6 ++ .../PhysicalTaskExecutorClientDelegator.java | 22 ++++- .../dispatcher/WorkerGroupDispatcher.java | 81 +++++++++++++++++-- .../WorkerGroupDispatcherCoordinator.java | 9 ++- .../WorkerGroupNotFoundException.java | 4 +- .../dispatch/WorkerNotFoundException.java | 25 ++++++ .../server/master/utils/ExceptionUtils.java | 10 +++ .../src/main/resources/application.yaml | 3 + .../dispatcher/WorkerGroupDispatcherTest.java | 4 +- .../plugin/task/api/TaskExecutionContext.java | 5 ++ 10 files changed, 152 insertions(+), 17 deletions(-) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java 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 220062bd5799..da99628bdb6e 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 @@ -74,6 +74,12 @@ public class MasterConfig implements Validator { */ private String masterRegistryPath; + /** + * Maximum time allowed for a task to be successfully dispatched. + * Default: 5 minutes. + */ + private Duration dispatchTimeout = Duration.ofMinutes(5); + @Override public boolean supports(Class clazz) { return MasterConfig.class.isAssignableFrom(clazz); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index f7566d3133b1..9de51915fc79 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -24,10 +24,13 @@ import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cluster.ClusterManager; import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; @@ -55,18 +58,29 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD @Autowired private IWorkerLoadBalancer workerLoadBalancer; + @Autowired + private ClusterManager clusterManager; + @Override public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); final String taskName = taskExecutionContext.getTaskName(); + final String workerGroup = taskExecutionContext.getWorkerGroup(); + if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { + throw new WorkerGroupNotFoundException( + String.format("Cannot find worker group to dispatch Task[id=%s, name=%s, workerGroup=%s]", + taskExecutionContext.getTaskInstanceId(), taskName, + workerGroup)); + } final String physicalTaskExecutorAddress = workerLoadBalancer - .select(taskExecutionContext.getWorkerGroup()) + .select(workerGroup) .map(Host::of) .map(Host::getAddress) - .orElseThrow(() -> new TaskDispatchException( - String.format("Cannot find the host to dispatch Task[id=%s, name=%s, workerGroup=%s]", + .orElseThrow(() -> new WorkerNotFoundException( + String.format( + "Cannot find available worker host to dispatch Task[id=%s, name=%s, workerGroup=%s]", taskExecutionContext.getTaskInstanceId(), taskName, - taskExecutionContext.getWorkerGroup()))); + workerGroup))); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 28834e27e7e1..f5d8a14b09f4 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -18,12 +18,19 @@ package org.apache.dolphinscheduler.server.master.engine.task.dispatcher; import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; +import java.time.Duration; +import java.util.Date; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,11 +55,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread { private final AtomicBoolean runningFlag = new AtomicBoolean(false); - public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient) { + private final Duration dispatchTimeout; + + public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient, + Duration dispatchTimeout) { super("WorkerGroupTaskDispatcher-" + workerGroupName); this.taskExecutorClient = taskExecutorClient; this.workerGroupEventBus = new TaskDispatchableEventBus<>(); this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet(); + this.dispatchTimeout = dispatchTimeout; log.info("Initialize WorkerGroupDispatcher: {}", this.getName()); } @@ -84,23 +95,77 @@ public void run() { } private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { + final int taskId = taskExecutionRunnable.getId(); + final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); + final long timeoutMs = this.dispatchTimeout.toMillis(); try { - if (!waitingDispatchTaskIds.remove(taskExecutionRunnable.getId())) { + if (!waitingDispatchTaskIds.remove(taskId)) { log.info( "The task: {} doesn't exist in waitingDispatchTaskIds(it might be paused or killed), will skip dispatch", - taskExecutionRunnable.getId()); + taskId); return; } taskExecutorClient.dispatch(taskExecutionRunnable); - } catch (Exception e) { + } catch (TaskDispatchException ex) { + // Checks whether the given task has exceeded its allowed dispatch timeout. + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + if (elapsed > timeoutMs) { + handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); + return; + } + // If dispatch failed, will put the task back to the queue // The task will be dispatched after waiting time. // the waiting time will increase multiple of times, but will not exceed 60 seconds - long waitingTimeMills = Math.min( + long waitingTimeMillis = Math.min( taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L); - dispatchTask(taskExecutionRunnable, waitingTimeMills); - log.error("Dispatch Task: {} failed will retry after: {}/ms", taskExecutionRunnable.getId(), - waitingTimeMills, e); + dispatchTask(taskExecutionRunnable, waitingTimeMillis); + log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskId, + waitingTimeMillis, ex); + } + } + + /** + * Marks the specified task as fatally failed due to an unrecoverable dispatch error,such as timeout or persistent client failure. + * Once this method is called, the task is considered permanently failed and will not be retried. + * + * @param taskExecutionRunnable the task to mark as fatally failed; must not be null + * @param exception the dispatch exception that triggered this failure handling; must not be null + * @param elapsed the time (in milliseconds) already spent attempting to dispatch the task + * @param timeoutMs the configured dispatch timeout threshold (in milliseconds) + */ + private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception, + long elapsed, long timeoutMs) { + int taskId = taskExecutionRunnable.getId(); + int workflowId = taskExecutionRunnable.getWorkflowInstance().getId(); + + log.warn("[DISPATCH_FAILED] taskId: {}, workflowId: {}, timed out after {} ms (limit: {} ms))", taskId, + workflowId, elapsed, timeoutMs); + + if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) { + log.error("[DISPATCH_FAILED] Worker group not found. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); + } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { + log.error("[DISPATCH_FAILED] No available workers. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); + } else { + log.error("[DISPATCH_FAILED] Unexpected dispatch error. taskId: {}, workflowId: {}", taskId, workflowId, + exception); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java index a85674c6f4d7..b3fea52976de 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.engine.task.dispatcher; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; @@ -41,8 +42,11 @@ public class WorkerGroupDispatcherCoordinator implements AutoCloseable { private final ConcurrentHashMap workerGroupDispatcherMap; - public WorkerGroupDispatcherCoordinator() { + private final MasterConfig masterConfig; + + public WorkerGroupDispatcherCoordinator(final MasterConfig masterConfig) { workerGroupDispatcherMap = new ConcurrentHashMap<>(); + this.masterConfig = masterConfig; } public void start() { @@ -99,7 +103,8 @@ public void close() throws Exception { private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) { return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> { - WorkerGroupDispatcher workerGroupDispatcher = new WorkerGroupDispatcher(wg, taskExecutorClient); + WorkerGroupDispatcher workerGroupDispatcher = + new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeout()); workerGroupDispatcher.start(); return workerGroupDispatcher; }); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java index c9d130172589..ddefa6fe0a04 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java @@ -19,7 +19,7 @@ public class WorkerGroupNotFoundException extends TaskDispatchException { - public WorkerGroupNotFoundException(String workerGroup) { - super("Cannot find worker group: " + workerGroup); + public WorkerGroupNotFoundException(String message) { + super(message); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java new file mode 100644 index 000000000000..10860ace2640 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java @@ -0,0 +1,25 @@ +/* + * 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.exception.dispatch; + +public class WorkerNotFoundException extends TaskDispatchException { + + public WorkerNotFoundException(String message) { + super(message); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index 9103bc5075d8..0997007057e6 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -17,6 +17,9 @@ package org.apache.dolphinscheduler.server.master.utils; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; + import org.springframework.dao.DataAccessResourceFailureException; public class ExceptionUtils { @@ -25,4 +28,11 @@ public static boolean isDatabaseConnectedFailedException(Throwable e) { return e instanceof DataAccessResourceFailureException; } + public static boolean isWorkerGroupNotFoundException(Throwable e) { + return e instanceof WorkerGroupNotFoundException; + } + + public static boolean isWorkerNotFoundException(Throwable e) { + return e instanceof WorkerNotFoundException; + } } diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 39a0f4311ab7..4980597de057 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -111,6 +111,9 @@ master: # Master max concurrent workflow instances, when the master's workflow instance count exceeds this value, master server will be marked as busy. max-concurrent-workflow-instances: 2147483647 worker-group-refresh-interval: 5m + # Timeout for task dispatching. If a task cannot be successfully dispatched to any worker within this duration, + # it will be marked as failed. Prevents tasks from being stuck indefinitely in the dispatch queue. + dispatch-timeout: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index 77525cb1810e..a7f4d3dba058 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; @@ -46,7 +47,8 @@ class WorkerGroupDispatcherTest { @BeforeEach void setUp() { taskExecutorClient = mock(ITaskExecutorClient.class); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient); + final MasterConfig masterConfig = new MasterConfig(); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeout()); } @Test diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java index a8f46435c1bb..fea0b50d32eb 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java @@ -126,6 +126,11 @@ public class TaskExecutionContext implements Serializable { private boolean failover; + /** + * Timestamp (ms) when the task was first enqueued for dispatch. + */ + private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis(); + public int increaseDispatchFailTimes() { return ++dispatchFailTimes; } From 1aa12df13380bb1058276dd131f843c424fa63b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 11:52:20 +0800 Subject: [PATCH 09/18] add dispatch-timeout-checker switch --- .../server/master/config/MasterConfig.java | 20 +++++++-- .../MasterDispatchTimeoutCheckerConfig.java | 42 +++++++++++++++++++ .../dispatcher/WorkerGroupDispatcher.java | 22 +++++----- .../WorkerGroupDispatcherCoordinator.java | 2 +- .../src/main/resources/application.yaml | 8 ++-- .../dispatcher/WorkerGroupDispatcherTest.java | 3 +- 6 files changed, 79 insertions(+), 18 deletions(-) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java 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 da99628bdb6e..ee8762c68378 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 @@ -75,10 +75,11 @@ public class MasterConfig implements Validator { private String masterRegistryPath; /** - * Maximum time allowed for a task to be successfully dispatched. - * Default: 5 minutes. + * Configuration for the master's task dispatch timeout check mechanism. + * This controls whether the system enforces a time limit for dispatching tasks to workers, + * and if so, how long to wait before marking a task as failed due to dispatch timeout. */ - private Duration dispatchTimeout = Duration.ofMinutes(5); + private MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker = new MasterDispatchTimeoutCheckerConfig(); @Override public boolean supports(Class clazz) { @@ -103,6 +104,19 @@ public void validate(Object target, Errors errors) { if (masterConfig.getWorkerGroupRefreshInterval().getSeconds() < 10) { errors.rejectValue("worker-group-refresh-interval", null, "should >= 10s"); } + + // Validate dispatch timeout checker config + MasterDispatchTimeoutCheckerConfig timeoutChecker = masterConfig.getDispatchTimeoutChecker(); + if (timeoutChecker != null && timeoutChecker.isEnabled()) { + if (timeoutChecker.getTimeoutDuration() == null) { + errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + "must be specified when dispatch timeout checker is enabled"); + } else if (timeoutChecker.getTimeoutDuration().toMillis() <= 0) { + errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + "must be a positive duration (e.g., '2m', '5m', '30m')"); + } + } + if (StringUtils.isEmpty(masterConfig.getMasterAddress())) { masterConfig.setMasterAddress(NetUtils.getAddr(masterConfig.getListenPort())); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java new file mode 100644 index 000000000000..aa83d1eba365 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java @@ -0,0 +1,42 @@ +/* + * 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.config; + +import java.time.Duration; + +import lombok.Data; + +/** + * Configuration for the master's task dispatch timeout checker. + * If enabled, tasks that remain in the dispatch queue longer than {@link #timeoutDuration} will be marked as failed to prevent indefinite queuing. + */ +@Data +public class MasterDispatchTimeoutCheckerConfig { + + /** + * Whether to enable the dispatch timeout checking mechanism. + */ + private boolean enabled = false; + + /** + * Maximum allowed time for a task to be dispatched to a worker. + * Tasks exceeding this duration in the dispatch queue will be failed. + * Examples: "2m", "5m", "30m". Defaults to 5 minutes. + */ + private Duration timeoutDuration = Duration.ofMinutes(5); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index f5d8a14b09f4..2fb5dd4d35d2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; @@ -29,7 +30,6 @@ import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; -import java.time.Duration; import java.util.Date; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -55,15 +55,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread { private final AtomicBoolean runningFlag = new AtomicBoolean(false); - private final Duration dispatchTimeout; + private final MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker; public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient, - Duration dispatchTimeout) { + MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker) { super("WorkerGroupTaskDispatcher-" + workerGroupName); this.taskExecutorClient = taskExecutorClient; this.workerGroupEventBus = new TaskDispatchableEventBus<>(); this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet(); - this.dispatchTimeout = dispatchTimeout; + this.dispatchTimeoutChecker = dispatchTimeoutChecker; log.info("Initialize WorkerGroupDispatcher: {}", this.getName()); } @@ -97,7 +97,6 @@ public void run() { private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { final int taskId = taskExecutionRunnable.getId(); final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); - final long timeoutMs = this.dispatchTimeout.toMillis(); try { if (!waitingDispatchTaskIds.remove(taskId)) { log.info( @@ -107,11 +106,14 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { } taskExecutorClient.dispatch(taskExecutionRunnable); } catch (TaskDispatchException ex) { - // Checks whether the given task has exceeded its allowed dispatch timeout. - long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); - if (elapsed > timeoutMs) { - handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); - return; + if (dispatchTimeoutChecker.isEnabled()) { + // Checks whether the given task has exceeded its allowed dispatch timeout. + long timeoutMs = this.dispatchTimeoutChecker.getTimeoutDuration().toMillis(); + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + if (elapsed > timeoutMs) { + handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); + return; + } } // If dispatch failed, will put the task back to the queue diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java index b3fea52976de..66663a923a1e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java @@ -104,7 +104,7 @@ public void close() throws Exception { private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) { return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> { WorkerGroupDispatcher workerGroupDispatcher = - new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeout()); + new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); workerGroupDispatcher.start(); return workerGroupDispatcher; }); diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 4980597de057..d3a62eced67e 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -111,9 +111,11 @@ master: # Master max concurrent workflow instances, when the master's workflow instance count exceeds this value, master server will be marked as busy. max-concurrent-workflow-instances: 2147483647 worker-group-refresh-interval: 5m - # Timeout for task dispatching. If a task cannot be successfully dispatched to any worker within this duration, - # it will be marked as failed. Prevents tasks from being stuck indefinitely in the dispatch queue. - dispatch-timeout: 5m + # Task dispatch timeout check (currently disabled). + # When enabled, tasks not dispatched within this duration are marked as failed. + dispatch-timeout-checker: + enabled: false + timeout-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index a7f4d3dba058..a60d87e50d6a 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -48,7 +48,8 @@ class WorkerGroupDispatcherTest { void setUp() { taskExecutorClient = mock(ITaskExecutorClient.class); final MasterConfig masterConfig = new MasterConfig(); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeout()); + dispatcher = + new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); } @Test From 0c29a82916fc3c070a98e1e122cd16d28745f9d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 16:19:10 +0800 Subject: [PATCH 10/18] add dispatch-timeout-checker switch --- .../server/master/config/MasterConfig.java | 9 ++++--- .../MasterDispatchTimeoutCheckerConfig.java | 4 +-- .../PhysicalTaskExecutorClientDelegator.java | 13 +++------ .../dispatcher/WorkerGroupDispatcher.java | 27 ++++++++----------- ...n.java => NoAvailableWorkerException.java} | 6 ++--- .../WorkerGroupNotFoundException.java | 4 +-- .../server/master/utils/ExceptionUtils.java | 1 - .../src/main/resources/application.yaml | 2 +- .../src/test/resources/application.yaml | 5 ++++ .../plugin/task/api/TaskExecutionContext.java | 4 +-- 10 files changed, 34 insertions(+), 41 deletions(-) rename dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/{WorkerNotFoundException.java => NoAvailableWorkerException.java} (80%) 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 ee8762c68378..b727ad508b79 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 @@ -108,11 +108,11 @@ public void validate(Object target, Errors errors) { // Validate dispatch timeout checker config MasterDispatchTimeoutCheckerConfig timeoutChecker = masterConfig.getDispatchTimeoutChecker(); if (timeoutChecker != null && timeoutChecker.isEnabled()) { - if (timeoutChecker.getTimeoutDuration() == null) { - errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + if (timeoutChecker.getMaxTaskDispatchDuration() == null) { + errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be specified when dispatch timeout checker is enabled"); - } else if (timeoutChecker.getTimeoutDuration().toMillis() <= 0) { - errors.rejectValue("master-dispatch-timeout-checker.timeout-duration", null, + } else if (timeoutChecker.getMaxTaskDispatchDuration().toMillis() <= 0) { + errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be a positive duration (e.g., '2m', '5m', '30m')"); } } @@ -142,6 +142,7 @@ private void printConfig() { "\n command-fetch-strategy: " + commandFetchStrategy + "\n worker-load-balancer-configuration-properties: " + workerLoadBalancerConfigurationProperties + + "\n dispatchTimeoutChecker: " + dispatchTimeoutChecker + "\n****************************Master Configuration**************************************"; log.info(config); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java index aa83d1eba365..0fc8467f47ce 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java @@ -23,7 +23,7 @@ /** * Configuration for the master's task dispatch timeout checker. - * If enabled, tasks that remain in the dispatch queue longer than {@link #timeoutDuration} will be marked as failed to prevent indefinite queuing. + * If enabled, tasks that remain in the dispatch queue longer than {@link #maxTaskDispatchDuration} will be marked as failed to prevent indefinite queuing. */ @Data public class MasterDispatchTimeoutCheckerConfig { @@ -38,5 +38,5 @@ public class MasterDispatchTimeoutCheckerConfig { * Tasks exceeding this duration in the dispatch queue will be failed. * Examples: "2m", "5m", "30m". Defaults to 5 minutes. */ - private Duration timeoutDuration = Duration.ofMinutes(5); + private Duration maxTaskDispatchDuration = Duration.ofMinutes(5); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 9de51915fc79..271c03f70a34 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -28,9 +28,9 @@ import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; -import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; @@ -67,20 +67,13 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws final String taskName = taskExecutionContext.getTaskName(); final String workerGroup = taskExecutionContext.getWorkerGroup(); if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { - throw new WorkerGroupNotFoundException( - String.format("Cannot find worker group to dispatch Task[id=%s, name=%s, workerGroup=%s]", - taskExecutionContext.getTaskInstanceId(), taskName, - workerGroup)); + throw new WorkerGroupNotFoundException(workerGroup); } final String physicalTaskExecutorAddress = workerLoadBalancer .select(workerGroup) .map(Host::of) .map(Host::getAddress) - .orElseThrow(() -> new WorkerNotFoundException( - String.format( - "Cannot find available worker host to dispatch Task[id=%s, name=%s, workerGroup=%s]", - taskExecutionContext.getTaskInstanceId(), taskName, - workerGroup))); + .orElseThrow(() -> new NoAvailableWorkerException(workerGroup)); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 2fb5dd4d35d2..0d8f03ad3008 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -95,21 +95,21 @@ public void run() { } private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { - final int taskId = taskExecutionRunnable.getId(); + final int taskInstanceId = taskExecutionRunnable.getId(); final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); try { - if (!waitingDispatchTaskIds.remove(taskId)) { + if (!waitingDispatchTaskIds.remove(taskInstanceId)) { log.info( "The task: {} doesn't exist in waitingDispatchTaskIds(it might be paused or killed), will skip dispatch", - taskId); + taskInstanceId); return; } taskExecutorClient.dispatch(taskExecutionRunnable); } catch (TaskDispatchException ex) { if (dispatchTimeoutChecker.isEnabled()) { // Checks whether the given task has exceeded its allowed dispatch timeout. - long timeoutMs = this.dispatchTimeoutChecker.getTimeoutDuration().toMillis(); - long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs(); + long timeoutMs = this.dispatchTimeoutChecker.getMaxTaskDispatchDuration().toMillis(); + long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchTime(); if (elapsed > timeoutMs) { handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); return; @@ -122,7 +122,7 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { long waitingTimeMillis = Math.min( taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L); dispatchTask(taskExecutionRunnable, waitingTimeMillis); - log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskId, + log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskInstanceId, waitingTimeMillis, ex); } } @@ -138,31 +138,26 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { */ private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception, long elapsed, long timeoutMs) { - int taskId = taskExecutionRunnable.getId(); - int workflowId = taskExecutionRunnable.getWorkflowInstance().getId(); + final String taskName = taskExecutionRunnable.getName(); - log.warn("[DISPATCH_FAILED] taskId: {}, workflowId: {}, timed out after {} ms (limit: {} ms))", taskId, - workflowId, elapsed, timeoutMs); + log.warn("[DISPATCH_FAILED] taskName: {}, timed out after {} ms (limit: {} ms))", taskName, elapsed, timeoutMs); if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) { - log.error("[DISPATCH_FAILED] Worker group not found. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, Worker group not found.", taskName, exception); final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { - log.error("[DISPATCH_FAILED] No available workers. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); } else { - log.error("[DISPATCH_FAILED] Unexpected dispatch error. taskId: {}, workflowId: {}", taskId, workflowId, - exception); + log.error("[DISPATCH_FAILED] taskName: {}, Unexpected dispatch error.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java similarity index 80% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java index 10860ace2640..d51e2342cc2b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/NoAvailableWorkerException.java @@ -17,9 +17,9 @@ package org.apache.dolphinscheduler.server.master.exception.dispatch; -public class WorkerNotFoundException extends TaskDispatchException { +public class NoAvailableWorkerException extends TaskDispatchException { - public WorkerNotFoundException(String message) { - super(message); + public NoAvailableWorkerException(String workerGroup) { + super("Cannot find available worker under worker group: " + workerGroup); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java index ddefa6fe0a04..c9d130172589 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/dispatch/WorkerGroupNotFoundException.java @@ -19,7 +19,7 @@ public class WorkerGroupNotFoundException extends TaskDispatchException { - public WorkerGroupNotFoundException(String message) { - super(message); + public WorkerGroupNotFoundException(String workerGroup) { + super("Cannot find worker group: " + workerGroup); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index 0997007057e6..fe70a69ec02c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -18,7 +18,6 @@ package org.apache.dolphinscheduler.server.master.utils; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; -import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException; import org.springframework.dao.DataAccessResourceFailureException; diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index d3a62eced67e..1f741559efb5 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -115,7 +115,7 @@ master: # When enabled, tasks not dispatched within this duration are marked as failed. dispatch-timeout-checker: enabled: false - timeout-duration: 5m + max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-master/src/test/resources/application.yaml b/dolphinscheduler-master/src/test/resources/application.yaml index 3aca1e3142eb..548a1a84291c 100644 --- a/dolphinscheduler-master/src/test/resources/application.yaml +++ b/dolphinscheduler-master/src/test/resources/application.yaml @@ -73,6 +73,11 @@ master: cpu-usage-weight: 30 task-thread-pool-usage-weight: 30 worker-group-refresh-interval: 5m + # Task dispatch timeout check (currently disabled). + # When enabled, tasks not dispatched within this duration are marked as failed. + dispatch-timeout-checker: + enabled: false + max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED config: diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java index fea0b50d32eb..e0fc55922640 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java @@ -127,9 +127,9 @@ public class TaskExecutionContext implements Serializable { private boolean failover; /** - * Timestamp (ms) when the task was first enqueued for dispatch. + * Timestamp (ms) when the task was first dispatched. */ - private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis(); + private final long firstDispatchTime = System.currentTimeMillis(); public int increaseDispatchFailTimes() { return ++dispatchFailTimes; From 6ad77d184101eaa69239de3005bbd99b01ca7555 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 16:45:39 +0800 Subject: [PATCH 11/18] update ExceptionUtils --- .../master/engine/task/dispatcher/WorkerGroupDispatcher.java | 2 +- .../dolphinscheduler/server/master/utils/ExceptionUtils.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 0d8f03ad3008..12e9acb457c9 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -149,7 +149,7 @@ private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); - } else if (ExceptionUtils.isWorkerNotFoundException(exception)) { + } else if (ExceptionUtils.isNoAvailableWorkerException(exception)) { log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, exception); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java index fe70a69ec02c..892dfab065e0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.utils; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; import org.springframework.dao.DataAccessResourceFailureException; @@ -31,7 +32,7 @@ public static boolean isWorkerGroupNotFoundException(Throwable e) { return e instanceof WorkerGroupNotFoundException; } - public static boolean isWorkerNotFoundException(Throwable e) { - return e instanceof WorkerNotFoundException; + public static boolean isNoAvailableWorkerException(Throwable e) { + return e instanceof NoAvailableWorkerException; } } From b760003fe3a8220de99298f9b8ac5b267e5e3931 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Wed, 17 Dec 2025 19:53:38 +0800 Subject: [PATCH 12/18] add test for dispatch timeout checker --- .../dispatcher/WorkerGroupDispatcherTest.java | 296 ++++++++++++++++++ 1 file changed, 296 insertions(+) diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index a60d87e50d6a..fb63676867fd 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -19,22 +19,37 @@ import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; +import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; import java.time.Duration; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.InOrder; @@ -141,4 +156,285 @@ void dispatch_TaskDispatchFails_RetryLogicWorks() throws TaskDispatchException { .untilAsserted(() -> verify(taskExecutorClient, times(2)).dispatch(taskExecutionRunnable)); } + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + WorkerGroupNotFoundException ex = new WorkerGroupNotFoundException("no worker group"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 500); + + WorkerGroupNotFoundException ex = new WorkerGroupNotFoundException("worker group not found"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFatalLifecycleEvent && + ((TaskFatalLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_WorkerGroupNotFound_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Dispatcher configured with a 5-minute timeout (enabled) + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + // Mock task with first dispatch time set to 100ms ago → well within timeout window + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect actual dispatch invocation + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub client to throw WorkerGroupNotFoundException and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Confirm dispatch was attempted + throw new WorkerGroupNotFoundException("Worker group 'TestGroup' does not exist"); + }).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + // When: Start dispatcher and dispatch the task + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Wait up to 1 second for the dispatch attempt to complete + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published because timeout has NOT been exceeded + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + NoAvailableWorkerException ex = new NoAvailableWorkerException("no worker"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given: enable timeout (200ms), task already waited 500ms + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = + mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis() - 500); + + NoAvailableWorkerException ex = new NoAvailableWorkerException("no worker"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && + ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_NoAvailableWorker_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Configure dispatcher with a 5-minute dispatch timeout (enabled) + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + // Mock task with first dispatch time set to 100ms ago → ensures it's NOT timed out yet + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect when dispatch is actually invoked (avoids timing flakiness) + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub the client to throw NoAvailableWorkerException on dispatch and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Signal that dispatch was attempted + throw new NoAvailableWorkerException("no worker"); + }).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + // When: Start dispatcher and trigger task dispatch + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Wait up to 1 second for the dispatch attempt to occur (ensures async execution completes) + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published since timeout has NOT been exceeded + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutDisabled_ShouldKeepRetrying() throws TaskDispatchException { + // Given + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis()); + TaskDispatchException ex = new TaskDispatchException("generic dispatch error"); + doThrow(ex).when(taskExecutorClient).dispatch(task); + + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // When & Then + await().atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + // Ensure it's retrying + verify(taskExecutorClient, atLeast(2)).dispatch(task); + + // Ensure NO event has been published during this time + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { + // Given + MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); + dispatchTimeoutCheckerConfig.setEnabled(true); + dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + + ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 500); + + TaskDispatchException ex = new TaskDispatchException("generic dispatch error"); + doThrow(ex).when(taskExecutorClient).dispatch(taskExecutionRunnable); + + dispatcher.start(); + dispatcher.dispatchTask(taskExecutionRunnable, 0); + + // Then + await().atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> { + verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); + WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && + ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + }); + } + + @Test + void dispatchTask_GenericTaskDispatchException_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { + // Given: Dispatcher configured with a 5-minute dispatch timeout (enabled) + MasterDispatchTimeoutCheckerConfig config = new MasterDispatchTimeoutCheckerConfig(); + config.setEnabled(true); + config.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, config); + + // Mock task with first dispatch time set to 100ms ago → well within timeout window + ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime( + System.currentTimeMillis() - 100); + + // Use CountDownLatch to reliably detect when dispatch is actually invoked + CountDownLatch dispatchCalled = new CountDownLatch(1); + + // Stub client to throw a generic TaskDispatchException and signal the latch + doAnswer(invocation -> { + dispatchCalled.countDown(); // Confirm dispatch attempt occurred + throw new TaskDispatchException("Generic dispatch error"); + }).when(taskExecutorClient).dispatch(task); + + // When: Start dispatcher and trigger task dispatch + dispatcher.start(); + dispatcher.dispatchTask(task, 0); + + // Wait up to 1 second for the dispatch attempt to complete (handles async execution) + boolean dispatched = dispatchCalled.await(1000, TimeUnit.MILLISECONDS); + Assertions.assertTrue(dispatched, "Expected dispatch() to be called within 1 second"); + + // Then: Verify NO failure events are published because timeout has NOT been exceeded + WorkflowEventBus eventBus = task.getWorkflowEventBus(); + verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); + } + + private ITaskExecutionRunnable mockTaskExecutionRunnableWithFirstDispatchTime(long firstDispatchTime) { + ITaskExecutionRunnable taskExecutionRunnable = mock(ITaskExecutionRunnable.class); + TaskInstance taskInstance = mock(TaskInstance.class); + WorkflowInstance workflowInstance = mock(WorkflowInstance.class); + WorkflowEventBus eventBus = mock(WorkflowEventBus.class); + + TaskExecutionContext context = mock(TaskExecutionContext.class); + when(context.getFirstDispatchTime()).thenReturn(firstDispatchTime); + + when(taskExecutionRunnable.getTaskInstance()).thenReturn(taskInstance); + when(taskExecutionRunnable.getWorkflowInstance()).thenReturn(workflowInstance); + when(taskExecutionRunnable.getWorkflowEventBus()).thenReturn(eventBus); + when(taskExecutionRunnable.getId()).thenReturn(ThreadLocalRandom.current().nextInt(1000, 9999)); + when(taskExecutionRunnable.getTaskExecutionContext()).thenReturn(context); + + return taskExecutionRunnable; + } } From 39b40c101d9c0d2e61fb506a185506b66cb34e32 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Fri, 19 Dec 2025 14:50:16 +0800 Subject: [PATCH 13/18] add TaskFatalLifecycleEvent and handler --- .../PhysicalTaskExecutorClientDelegator.java | 4 +++ .../lifecycle/TaskLifecycleEventType.java | 4 +++ .../statemachine/AbstractTaskStateAction.java | 33 +++++++++++++++++++ .../task/statemachine/ITaskStateAction.java | 9 +++++ 4 files changed, 50 insertions(+) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 271c03f70a34..96331b2dcae9 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -66,9 +66,13 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); final String taskName = taskExecutionContext.getTaskName(); final String workerGroup = taskExecutionContext.getWorkerGroup(); + + // workerGroup not exist if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) { throw new WorkerGroupNotFoundException(workerGroup); } + + // select an available worker from the worker group; throws NoAvailableWorkerException if none is available. final String physicalTaskExecutorAddress = workerLoadBalancer .select(workerGroup) .map(Host::of) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java index 5ddcf13c8917..fb12ccb60375 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java @@ -29,6 +29,10 @@ public enum TaskLifecycleEventType implements ILifecycleEventType { * Dispatch the task instance to target. */ DISPATCH, + /** + * Task instance encounters catastrophic failure(such as initialization failure), it will enter a failed state. + */ + FATAL, /** * The task instance is dispatched to the target executor server. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java index ad0e652d4866..738f284b9a7b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java @@ -36,6 +36,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; @@ -99,6 +100,38 @@ protected void releaseTaskInstanceResourcesIfNeeded(final ITaskExecutionRunnable } } + @Override + public void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + releaseTaskInstanceResourcesIfNeeded(taskExecutionRunnable); + persistentTaskInstanceFatalEventToDB(taskExecutionRunnable, taskFatalEvent); + + if (taskExecutionRunnable.isTaskInstanceCanRetry()) { + taskExecutionRunnable.getWorkflowEventBus().publish(TaskRetryLifecycleEvent.of(taskExecutionRunnable)); + return; + } + + // If all successors are condition tasks, then the task will not be marked as failure. + // And the DAG will continue to execute. + final IWorkflowExecutionGraph workflowExecutionGraph = taskExecutionRunnable.getWorkflowExecutionGraph(); + if (workflowExecutionGraph.isAllSuccessorsAreConditionTask(taskExecutionRunnable)) { + mergeTaskVarPoolToWorkflow(workflowExecutionRunnable, taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + return; + } + taskExecutionRunnable.getWorkflowExecutionGraph().markTaskExecutionRunnableChainFailure(taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + } + + private void persistentTaskInstanceFatalEventToDB(final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + taskInstance.setState(TaskExecutionStatus.FAILURE); + taskInstance.setEndTime(taskFatalEvent.getEndTime()); + taskInstanceDao.updateById(taskInstance); + } + @Override public void onDispatchedEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java index a041de5e3ca6..f60c3ae13532 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailoverLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; @@ -91,6 +92,14 @@ void onDispatchEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskDispatchLifecycleEvent taskDispatchEvent); + /** + * Perform the necessary actions when the task in a certain state receive a {@link TaskFatalLifecycleEvent}. + *

This method is called when the task encounters catastrophic failure (e.g., initialization failure). + */ + void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent); + /** * Perform the necessary actions when the task in a certain state receive a {@link TaskDispatchedLifecycleEvent}. *

This method is called when the task has been dispatched to executor. From d7bf4fa30af622790a441bf571175213c19c53dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Fri, 19 Dec 2025 14:50:53 +0800 Subject: [PATCH 14/18] add TaskFatalLifecycleEvent and handler --- .../event/TaskFatalLifecycleEvent.java | 52 +++++++++++++++++++ .../TaskFatalLifecycleEventHandler.java | 44 ++++++++++++++++ 2 files changed, 96 insertions(+) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java new file mode 100644 index 000000000000..88cfc9a478f6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java @@ -0,0 +1,52 @@ +/* + * 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.engine.task.lifecycle.event; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Date; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +@AllArgsConstructor +public class TaskFatalLifecycleEvent extends AbstractTaskLifecycleEvent { + + private final ITaskExecutionRunnable taskExecutionRunnable; + + private final Date endTime; + + @Override + public ILifecycleEventType getEventType() { + return TaskLifecycleEventType.FATAL; + } + + @Override + public String toString() { + return "TaskFatalLifecycleEvent{" + + "task=" + taskExecutionRunnable.getName() + + ", endTime=" + endTime + + '}'; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java new file mode 100644 index 000000000000..8277d8b2caa7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java @@ -0,0 +1,44 @@ +/* + * 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.engine.task.lifecycle.handler; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; +import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; + +import org.springframework.stereotype.Component; + +@Component +public class TaskFatalLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + + @Override + public void handle(final ITaskStateAction taskStateAction, + final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + taskStateAction.onFatalEvent(workflowExecutionRunnable, taskExecutionRunnable, taskFatalEvent); + } + + @Override + public ILifecycleEventType matchEventType() { + return TaskLifecycleEventType.FATAL; + } +} From 5b2722dd594a2f1580bea5f8309cd3859be4dd78 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Mon, 22 Dec 2025 17:54:33 +0800 Subject: [PATCH 15/18] update TaskDispatchPolicy --- .../server/master/config/MasterConfig.java | 14 ++--- ...kerConfig.java => TaskDispatchPolicy.java} | 24 +++++---- .../dispatcher/WorkerGroupDispatcher.java | 34 ++++++------ .../WorkerGroupDispatcherCoordinator.java | 2 +- .../lifecycle/TaskLifecycleEventType.java | 4 -- .../event/TaskFatalLifecycleEvent.java | 52 ------------------- .../TaskFatalLifecycleEventHandler.java | 44 ---------------- .../statemachine/AbstractTaskStateAction.java | 33 ------------ .../task/statemachine/ITaskStateAction.java | 9 ---- .../src/main/resources/application.yaml | 4 +- .../dispatcher/WorkerGroupDispatcherTest.java | 42 ++++++--------- .../src/test/resources/application.yaml | 4 +- 12 files changed, 59 insertions(+), 207 deletions(-) rename dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/{MasterDispatchTimeoutCheckerConfig.java => TaskDispatchPolicy.java} (55%) delete mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java delete mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java 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 b727ad508b79..a801e6311c0f 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 @@ -79,7 +79,7 @@ public class MasterConfig implements Validator { * This controls whether the system enforces a time limit for dispatching tasks to workers, * and if so, how long to wait before marking a task as failed due to dispatch timeout. */ - private MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker = new MasterDispatchTimeoutCheckerConfig(); + private TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); @Override public boolean supports(Class clazz) { @@ -105,13 +105,13 @@ public void validate(Object target, Errors errors) { errors.rejectValue("worker-group-refresh-interval", null, "should >= 10s"); } - // Validate dispatch timeout checker config - MasterDispatchTimeoutCheckerConfig timeoutChecker = masterConfig.getDispatchTimeoutChecker(); - if (timeoutChecker != null && timeoutChecker.isEnabled()) { - if (timeoutChecker.getMaxTaskDispatchDuration() == null) { + // Validate task dispatch policy config + TaskDispatchPolicy configTaskDispatchPolicy = masterConfig.getTaskDispatchPolicy(); + if (configTaskDispatchPolicy != null && configTaskDispatchPolicy.isDispatchTimeoutFailedEnabled()) { + if (configTaskDispatchPolicy.getMaxTaskDispatchDuration() == null) { errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be specified when dispatch timeout checker is enabled"); - } else if (timeoutChecker.getMaxTaskDispatchDuration().toMillis() <= 0) { + } else if (configTaskDispatchPolicy.getMaxTaskDispatchDuration().toMillis() <= 0) { errors.rejectValue("dispatch-timeout-checker.max-task-dispatch-duration", null, "must be a positive duration (e.g., '2m', '5m', '30m')"); } @@ -142,7 +142,7 @@ private void printConfig() { "\n command-fetch-strategy: " + commandFetchStrategy + "\n worker-load-balancer-configuration-properties: " + workerLoadBalancerConfigurationProperties + - "\n dispatchTimeoutChecker: " + dispatchTimeoutChecker + + "\n taskDispatchPolicy: " + taskDispatchPolicy + "\n****************************Master Configuration**************************************"; log.info(config); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/TaskDispatchPolicy.java similarity index 55% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/TaskDispatchPolicy.java index 0fc8467f47ce..5be6a7041b3b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterDispatchTimeoutCheckerConfig.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/TaskDispatchPolicy.java @@ -22,21 +22,27 @@ import lombok.Data; /** - * Configuration for the master's task dispatch timeout checker. - * If enabled, tasks that remain in the dispatch queue longer than {@link #maxTaskDispatchDuration} will be marked as failed to prevent indefinite queuing. + * Configuration for the master's task dispatch policy. + *

+ * When enabled, tasks that remain in the dispatch queue longer than + * {@link #maxTaskDispatchDuration} will be marked as failed to prevent indefinite queuing. */ @Data -public class MasterDispatchTimeoutCheckerConfig { +public class TaskDispatchPolicy { /** - * Whether to enable the dispatch timeout checking mechanism. + * Indicates whether the dispatch timeout checking mechanism is enabled. + *

+ * If {@code true}, tasks exceeding the configured dispatch duration will be failed automatically. */ - private boolean enabled = false; + private boolean dispatchTimeoutFailedEnabled = false; /** - * Maximum allowed time for a task to be dispatched to a worker. - * Tasks exceeding this duration in the dispatch queue will be failed. - * Examples: "2m", "5m", "30m". Defaults to 5 minutes. + * The maximum allowed duration a task may wait in the dispatch queue before being assigned to a worker. + *

+ * Tasks that exceed this duration will be marked as failed. + *

+ * Examples: {@code "2m"}, {@code "5m"}, {@code "30m"}. */ - private Duration maxTaskDispatchDuration = Duration.ofMinutes(5); + private Duration maxTaskDispatchDuration; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index 12e9acb457c9..fc9f52cc395b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -20,13 +20,11 @@ import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; +import org.apache.dolphinscheduler.server.master.config.TaskDispatchPolicy; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; @@ -55,15 +53,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread { private final AtomicBoolean runningFlag = new AtomicBoolean(false); - private final MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker; + private final TaskDispatchPolicy taskDispatchPolicy; public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient, - MasterDispatchTimeoutCheckerConfig dispatchTimeoutChecker) { + TaskDispatchPolicy taskDispatchPolicy) { super("WorkerGroupTaskDispatcher-" + workerGroupName); this.taskExecutorClient = taskExecutorClient; this.workerGroupEventBus = new TaskDispatchableEventBus<>(); this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet(); - this.dispatchTimeoutChecker = dispatchTimeoutChecker; + this.taskDispatchPolicy = taskDispatchPolicy; log.info("Initialize WorkerGroupDispatcher: {}", this.getName()); } @@ -105,10 +103,10 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { return; } taskExecutorClient.dispatch(taskExecutionRunnable); - } catch (TaskDispatchException ex) { - if (dispatchTimeoutChecker.isEnabled()) { + } catch (Exception ex) { + if (taskDispatchPolicy.isDispatchTimeoutFailedEnabled()) { // Checks whether the given task has exceeded its allowed dispatch timeout. - long timeoutMs = this.dispatchTimeoutChecker.getMaxTaskDispatchDuration().toMillis(); + long timeoutMs = this.taskDispatchPolicy.getMaxTaskDispatchDuration().toMillis(); long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchTime(); if (elapsed > timeoutMs) { handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs); @@ -132,32 +130,32 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { * Once this method is called, the task is considered permanently failed and will not be retried. * * @param taskExecutionRunnable the task to mark as fatally failed; must not be null - * @param exception the dispatch exception that triggered this failure handling; must not be null + * @param ex the dispatch exception that triggered this failure handling; must not be null * @param elapsed the time (in milliseconds) already spent attempting to dispatch the task * @param timeoutMs the configured dispatch timeout threshold (in milliseconds) */ - private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception, + private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, Exception ex, long elapsed, long timeoutMs) { final String taskName = taskExecutionRunnable.getName(); log.warn("[DISPATCH_FAILED] taskName: {}, timed out after {} ms (limit: {} ms))", taskName, elapsed, timeoutMs); - if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) { - log.error("[DISPATCH_FAILED] taskName: {}, Worker group not found.", taskName, exception); - final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() + if (ExceptionUtils.isWorkerGroupNotFoundException(ex)) { + log.error("[DISPATCH_FAILED] taskName: {}, Worker group not found.", taskName, ex); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); - taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); - } else if (ExceptionUtils.isNoAvailableWorkerException(exception)) { - log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, exception); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); + } else if (ExceptionUtils.isNoAvailableWorkerException(ex)) { + log.error("[DISPATCH_FAILED] taskName: {}, No available worker.", taskName, ex); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) .build(); taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); } else { - log.error("[DISPATCH_FAILED] taskName: {}, Unexpected dispatch error.", taskName, exception); + log.error("[DISPATCH_FAILED] taskName: {}, Unexpected dispatch error.", taskName, ex); final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() .taskExecutionRunnable(taskExecutionRunnable) .endTime(new Date()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java index 66663a923a1e..086fc5359ee4 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinator.java @@ -104,7 +104,7 @@ public void close() throws Exception { private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) { return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> { WorkerGroupDispatcher workerGroupDispatcher = - new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); + new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getTaskDispatchPolicy()); workerGroupDispatcher.start(); return workerGroupDispatcher; }); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java index fb12ccb60375..5ddcf13c8917 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java @@ -29,10 +29,6 @@ public enum TaskLifecycleEventType implements ILifecycleEventType { * Dispatch the task instance to target. */ DISPATCH, - /** - * Task instance encounters catastrophic failure(such as initialization failure), it will enter a failed state. - */ - FATAL, /** * The task instance is dispatched to the target executor server. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java deleted file mode 100644 index 88cfc9a478f6..000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java +++ /dev/null @@ -1,52 +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.engine.task.lifecycle.event; - -import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; - -import java.util.Date; - -import lombok.AllArgsConstructor; -import lombok.Builder; -import lombok.Data; - -@Data -@Builder -@AllArgsConstructor -public class TaskFatalLifecycleEvent extends AbstractTaskLifecycleEvent { - - private final ITaskExecutionRunnable taskExecutionRunnable; - - private final Date endTime; - - @Override - public ILifecycleEventType getEventType() { - return TaskLifecycleEventType.FATAL; - } - - @Override - public String toString() { - return "TaskFatalLifecycleEvent{" + - "task=" + taskExecutionRunnable.getName() + - ", endTime=" + endTime + - '}'; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java deleted file mode 100644 index 8277d8b2caa7..000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java +++ /dev/null @@ -1,44 +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.engine.task.lifecycle.handler; - -import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; -import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; - -import org.springframework.stereotype.Component; - -@Component -public class TaskFatalLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { - - @Override - public void handle(final ITaskStateAction taskStateAction, - final IWorkflowExecutionRunnable workflowExecutionRunnable, - final ITaskExecutionRunnable taskExecutionRunnable, - final TaskFatalLifecycleEvent taskFatalEvent) { - taskStateAction.onFatalEvent(workflowExecutionRunnable, taskExecutionRunnable, taskFatalEvent); - } - - @Override - public ILifecycleEventType matchEventType() { - return TaskLifecycleEventType.FATAL; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java index 738f284b9a7b..ad0e652d4866 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java @@ -36,7 +36,6 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; @@ -100,38 +99,6 @@ protected void releaseTaskInstanceResourcesIfNeeded(final ITaskExecutionRunnable } } - @Override - public void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, - final ITaskExecutionRunnable taskExecutionRunnable, - final TaskFatalLifecycleEvent taskFatalEvent) { - releaseTaskInstanceResourcesIfNeeded(taskExecutionRunnable); - persistentTaskInstanceFatalEventToDB(taskExecutionRunnable, taskFatalEvent); - - if (taskExecutionRunnable.isTaskInstanceCanRetry()) { - taskExecutionRunnable.getWorkflowEventBus().publish(TaskRetryLifecycleEvent.of(taskExecutionRunnable)); - return; - } - - // If all successors are condition tasks, then the task will not be marked as failure. - // And the DAG will continue to execute. - final IWorkflowExecutionGraph workflowExecutionGraph = taskExecutionRunnable.getWorkflowExecutionGraph(); - if (workflowExecutionGraph.isAllSuccessorsAreConditionTask(taskExecutionRunnable)) { - mergeTaskVarPoolToWorkflow(workflowExecutionRunnable, taskExecutionRunnable); - publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); - return; - } - taskExecutionRunnable.getWorkflowExecutionGraph().markTaskExecutionRunnableChainFailure(taskExecutionRunnable); - publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); - } - - private void persistentTaskInstanceFatalEventToDB(final ITaskExecutionRunnable taskExecutionRunnable, - final TaskFatalLifecycleEvent taskFatalEvent) { - final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); - taskInstance.setState(TaskExecutionStatus.FAILURE); - taskInstance.setEndTime(taskFatalEvent.getEndTime()); - taskInstanceDao.updateById(taskInstance); - } - @Override public void onDispatchedEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java index f60c3ae13532..a041de5e3ca6 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java @@ -22,7 +22,6 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailoverLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; @@ -92,14 +91,6 @@ void onDispatchEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskDispatchLifecycleEvent taskDispatchEvent); - /** - * Perform the necessary actions when the task in a certain state receive a {@link TaskFatalLifecycleEvent}. - *

This method is called when the task encounters catastrophic failure (e.g., initialization failure). - */ - void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, - final ITaskExecutionRunnable taskExecutionRunnable, - final TaskFatalLifecycleEvent taskFatalEvent); - /** * Perform the necessary actions when the task in a certain state receive a {@link TaskDispatchedLifecycleEvent}. *

This method is called when the task has been dispatched to executor. diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 1f741559efb5..3d5c6685140b 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -113,8 +113,8 @@ master: worker-group-refresh-interval: 5m # Task dispatch timeout check (currently disabled). # When enabled, tasks not dispatched within this duration are marked as failed. - dispatch-timeout-checker: - enabled: false + task-dispatch-policy: + dispatch-timeout-failed-enabled: false max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index fb63676867fd..a6e3a793c7bf 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -34,11 +34,10 @@ import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.config.MasterDispatchTimeoutCheckerConfig; +import org.apache.dolphinscheduler.server.master.config.TaskDispatchPolicy; import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.dispatch.NoAvailableWorkerException; import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; @@ -64,7 +63,7 @@ void setUp() { taskExecutorClient = mock(ITaskExecutorClient.class); final MasterConfig masterConfig = new MasterConfig(); dispatcher = - new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeoutChecker()); + new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getTaskDispatchPolicy()); } @Test @@ -175,15 +174,14 @@ void dispatchTask_WorkerGroupNotFound_TimeoutDisabled_ShouldKeepRetrying() throw // Ensure NO event has been published during this time WorkflowEventBus eventBus = task.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); }); } @Test void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given - MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); - dispatchTimeoutCheckerConfig.setEnabled(true); + TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); + dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); @@ -202,17 +200,16 @@ void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFai .untilAsserted(() -> { verify(taskExecutorClient, times(1)).dispatch(taskExecutionRunnable); WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); - verify(eventBus).publish(argThat(evt -> evt instanceof TaskFatalLifecycleEvent && - ((TaskFatalLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); - verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); + verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && + ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); }); } @Test void dispatchTask_WorkerGroupNotFound_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Dispatcher configured with a 5-minute timeout (enabled) - MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); - dispatchTimeoutCheckerConfig.setEnabled(true); + TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); + dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); @@ -241,7 +238,6 @@ void dispatchTask_WorkerGroupNotFound_TimeoutEnabledButNotExceeded_ShouldNotPubl // Then: Verify NO failure events are published because timeout has NOT been exceeded WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); } @Test @@ -263,15 +259,14 @@ void dispatchTask_NoAvailableWorker_TimeoutDisabled_ShouldKeepRetrying() throws // Ensure NO event has been published during this time WorkflowEventBus eventBus = task.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); }); } @Test void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given: enable timeout (200ms), task already waited 500ms - MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); - dispatchTimeoutCheckerConfig.setEnabled(true); + TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); + dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); @@ -292,15 +287,14 @@ void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFaile WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); }); } @Test void dispatchTask_NoAvailableWorker_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Configure dispatcher with a 5-minute dispatch timeout (enabled) - MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); - dispatchTimeoutCheckerConfig.setEnabled(true); + TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); + dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); @@ -329,7 +323,6 @@ void dispatchTask_NoAvailableWorker_TimeoutEnabledButNotExceeded_ShouldNotPublis // Then: Verify NO failure events are published since timeout has NOT been exceeded WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); } @Test @@ -351,15 +344,14 @@ void dispatchTask_GenericTaskDispatchException_TimeoutDisabled_ShouldKeepRetryin // Ensure NO event has been published during this time WorkflowEventBus eventBus = task.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); }); } @Test void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given - MasterDispatchTimeoutCheckerConfig dispatchTimeoutCheckerConfig = new MasterDispatchTimeoutCheckerConfig(); - dispatchTimeoutCheckerConfig.setEnabled(true); + TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); + dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); @@ -380,15 +372,14 @@ void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldP WorkflowEventBus eventBus = taskExecutionRunnable.getWorkflowEventBus(); verify(eventBus).publish(argThat(evt -> evt instanceof TaskFailedLifecycleEvent && ((TaskFailedLifecycleEvent) evt).getTaskExecutionRunnable() == taskExecutionRunnable)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); }); } @Test void dispatchTask_GenericTaskDispatchException_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Dispatcher configured with a 5-minute dispatch timeout (enabled) - MasterDispatchTimeoutCheckerConfig config = new MasterDispatchTimeoutCheckerConfig(); - config.setEnabled(true); + TaskDispatchPolicy config = new TaskDispatchPolicy(); + config.setDispatchTimeoutFailedEnabled(true); config.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, config); @@ -417,7 +408,6 @@ void dispatchTask_GenericTaskDispatchException_TimeoutEnabledButNotExceeded_Shou // Then: Verify NO failure events are published because timeout has NOT been exceeded WorkflowEventBus eventBus = task.getWorkflowEventBus(); verify(eventBus, never()).publish(any(TaskFailedLifecycleEvent.class)); - verify(eventBus, never()).publish(any(TaskFatalLifecycleEvent.class)); } private ITaskExecutionRunnable mockTaskExecutionRunnableWithFirstDispatchTime(long firstDispatchTime) { diff --git a/dolphinscheduler-master/src/test/resources/application.yaml b/dolphinscheduler-master/src/test/resources/application.yaml index 548a1a84291c..b03b815a5b8a 100644 --- a/dolphinscheduler-master/src/test/resources/application.yaml +++ b/dolphinscheduler-master/src/test/resources/application.yaml @@ -75,8 +75,8 @@ master: worker-group-refresh-interval: 5m # Task dispatch timeout check (currently disabled). # When enabled, tasks not dispatched within this duration are marked as failed. - dispatch-timeout-checker: - enabled: false + task-dispatch-policy: + dispatch-timeout-failed-enabled: false max-task-dispatch-duration: 5m command-fetch-strategy: type: ID_SLOT_BASED From 33c16e8b9473450ec58fba7bb62af5309032704c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Tue, 23 Dec 2025 09:50:32 +0800 Subject: [PATCH 16/18] add it test case --- .../dispatcher/WorkerGroupDispatcher.java | 3 +- .../cases/WorkflowStartTestCase.java | 87 +++++++++++++++++++ .../workflow_with_worker_group_not_found.yaml | 61 +++++++++++++ 3 files changed, 150 insertions(+), 1 deletion(-) create mode 100644 dolphinscheduler-master/src/test/resources/it/start/workflow_with_worker_group_not_found.yaml diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java index fc9f52cc395b..1c1e7f6664d1 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcher.java @@ -106,6 +106,7 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { } catch (Exception ex) { if (taskDispatchPolicy.isDispatchTimeoutFailedEnabled()) { // Checks whether the given task has exceeded its allowed dispatch timeout. + // If a dispatch timeout occurs, the task will NOT be put back into the queue. long timeoutMs = this.taskDispatchPolicy.getMaxTaskDispatchDuration().toMillis(); long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchTime(); if (elapsed > timeoutMs) { @@ -130,7 +131,7 @@ private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) { * Once this method is called, the task is considered permanently failed and will not be retried. * * @param taskExecutionRunnable the task to mark as fatally failed; must not be null - * @param ex the dispatch exception that triggered this failure handling; must not be null + * @param ex the dispatch exception that triggered this failure handling; must not be null * @param elapsed the time (in milliseconds) already spent attempting to dispatch the task * @param timeoutMs the configured dispatch timeout threshold (in milliseconds) */ diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java index a2f0331f3483..3010a2b4d937 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java @@ -33,6 +33,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property; import org.apache.dolphinscheduler.plugin.task.api.utils.VarPoolUtils; import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase; +import org.apache.dolphinscheduler.server.master.config.TaskDispatchPolicy; import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator; import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext; @@ -1435,4 +1436,90 @@ void testStartWorkflow_with_oneForbiddenConditionTaskWithOneFakePredecessor_runF }); masterContainer.assertAllResourceReleased(); } + + @Test + @DisplayName("Test start a workflow whose task specifies a non-existent worker group when dispatch timeout is enabled") + public void testStartWorkflow_with_workerGroupNotFoundAndTimeoutEnabled() { + // Enable dispatch timeout to ensure tasks fail fast if worker group is missing + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofSeconds(10)); + this.masterConfig.setTaskDispatchPolicy(taskDispatchPolicy); + + final String yaml = "/it/start/workflow_with_worker_group_not_found.yaml"; + final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml); + final WorkflowDefinition workflow = context.getOneWorkflow(); + + final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() + .workflowDefinition(workflow) + .build(); + + workflowOperator.manualTriggerWorkflow(workflowTriggerDTO); + + // Observe the task over a reasonable period (e.g., 20 seconds) + // It should reach a fail state because: + // - workerGroup "workerGroupNotFound" does not exist + // - and timeout detection is ON → fallback failure mechanism + await() + .atMost(Duration.ofSeconds(30)) + .untilAsserted(() -> { + Assertions + .assertThat(repository.queryWorkflowInstance(workflow)) + .satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState()) + .isEqualTo(WorkflowExecutionStatus.FAILURE)); + + Assertions + .assertThat(repository.queryTaskInstance(workflow)) + .hasSize(1) + .anySatisfy(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("A"); + assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE); + }); + }); + + masterContainer.assertAllResourceReleased(); + } + + @Test + @DisplayName("Task with non-existent worker group remains running indefinitely when dispatch timeout is disabled") + public void testTaskStaysRunning_whenWorkerGroupNotFoundAndTimeoutDisabled() { + // Disable dispatch timeout: system will NOT auto-fail tasks that cannot be dispatched + TaskDispatchPolicy policy = new TaskDispatchPolicy(); + policy.setDispatchTimeoutFailedEnabled(false); + this.masterConfig.setTaskDispatchPolicy(policy); + + final String yaml = "/it/start/workflow_with_worker_group_not_found.yaml"; + final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml); + final WorkflowDefinition workflow = context.getOneWorkflow(); + + final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() + .workflowDefinition(workflow) + .build(); + + workflowOperator.manualTriggerWorkflow(workflowTriggerDTO); + + // Observe the task over a reasonable period (e.g., 20 seconds) + // It should NEVER reach a terminal state because: + // - workerGroup "workerGroupNotFound" does not exist + // - and timeout detection is OFF → no fallback failure mechanism + await() + .atMost(Duration.ofSeconds(30)) + .untilAsserted(() -> { + Assertions + .assertThat(repository.queryWorkflowInstance(workflow)) + .satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState()) + .isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION)); + + Assertions + .assertThat(repository.queryTaskInstance(workflow)) + .hasSize(1) + .anySatisfy(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("A"); + assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION); + }); + }); + + masterContainer.assertAllResourceReleased(); + } + } diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_worker_group_not_found.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_worker_group_not_found.yaml new file mode 100644 index 000000000000..a2d5b3240502 --- /dev/null +++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_worker_group_not_found.yaml @@ -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. +# + +project: + name: MasterIntegrationTest + code: 1 + description: This is a fake project + userId: 1 + userName: admin + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + +workflows: + - name: workflow_with_worker_group_not_found + code: 1 + version: 1 + projectCode: 1 + description: This is a fake workflow with single task + releaseState: ONLINE + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + userId: 1 + executionType: PARALLEL + +tasks: + - name: A + code: 1 + version: 1 + projectCode: 1 + userId: 1 + taskType: LogicFakeTask + taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}' + workerGroup: workerGroupNotFound + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + taskExecuteType: BATCH + +taskRelations: + - projectCode: 1 + workflowDefinitionCode: 1 + workflowDefinitionVersion: 1 + preTaskCode: 0 + preTaskVersion: 0 + postTaskCode: 1 + postTaskVersion: 1 + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 From 99a5d7cd1cdb3370cbc61d9ef21e57a9283895f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Tue, 23 Dec 2025 10:42:30 +0800 Subject: [PATCH 17/18] update test --- .../dispatcher/WorkerGroupDispatcherTest.java | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java index a6e3a793c7bf..4f1def750758 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherTest.java @@ -180,11 +180,11 @@ void dispatchTask_WorkerGroupNotFound_TimeoutDisabled_ShouldKeepRetrying() throw @Test void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given - TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); - dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); - dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMillis(200)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( System.currentTimeMillis() - 500); @@ -208,11 +208,11 @@ void dispatchTask_WorkerGroupNotFound_TimeoutEnabledAndExceeded_ShouldPublishFai @Test void dispatchTask_WorkerGroupNotFound_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Dispatcher configured with a 5-minute timeout (enabled) - TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); - dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); - dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); // Mock task with first dispatch time set to 100ms ago → well within timeout window ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( @@ -265,11 +265,11 @@ void dispatchTask_NoAvailableWorker_TimeoutDisabled_ShouldKeepRetrying() throws @Test void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given: enable timeout (200ms), task already waited 500ms - TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); - dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); - dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMillis(200)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime(System.currentTimeMillis() - 500); @@ -293,11 +293,11 @@ void dispatchTask_NoAvailableWorker_TimeoutEnabledAndExceeded_ShouldPublishFaile @Test void dispatchTask_NoAvailableWorker_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Configure dispatcher with a 5-minute dispatch timeout (enabled) - TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); - dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); - dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); // Mock task with first dispatch time set to 100ms ago → ensures it's NOT timed out yet ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( @@ -350,11 +350,11 @@ void dispatchTask_GenericTaskDispatchException_TimeoutDisabled_ShouldKeepRetryin @Test void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldPublishFailedEvent() throws TaskDispatchException { // Given - TaskDispatchPolicy dispatchTimeoutCheckerConfig = new TaskDispatchPolicy(); - dispatchTimeoutCheckerConfig.setDispatchTimeoutFailedEnabled(true); - dispatchTimeoutCheckerConfig.setMaxTaskDispatchDuration(Duration.ofMillis(200)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMillis(200)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, dispatchTimeoutCheckerConfig); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); ITaskExecutionRunnable taskExecutionRunnable = mockTaskExecutionRunnableWithFirstDispatchTime( System.currentTimeMillis() - 500); @@ -378,11 +378,11 @@ void dispatchTask_GenericTaskDispatchException_TimeoutEnabledAndExceeded_ShouldP @Test void dispatchTask_GenericTaskDispatchException_TimeoutEnabledButNotExceeded_ShouldNotPublishAnyFailureEvent() throws TaskDispatchException, InterruptedException { // Given: Dispatcher configured with a 5-minute dispatch timeout (enabled) - TaskDispatchPolicy config = new TaskDispatchPolicy(); - config.setDispatchTimeoutFailedEnabled(true); - config.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); + TaskDispatchPolicy taskDispatchPolicy = new TaskDispatchPolicy(); + taskDispatchPolicy.setDispatchTimeoutFailedEnabled(true); + taskDispatchPolicy.setMaxTaskDispatchDuration(Duration.ofMinutes(5)); - dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, config); + dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, taskDispatchPolicy); // Mock task with first dispatch time set to 100ms ago → well within timeout window ITaskExecutionRunnable task = mockTaskExecutionRunnableWithFirstDispatchTime( From 879ba5e8961ef1717be2696677a9744539bf3bb7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8B=8F=E4=B9=89=E8=B6=85?= Date: Thu, 8 Jan 2026 10:45:08 +0800 Subject: [PATCH 18/18] update test --- .../WorkerGroupDispatcherCoordinatorTest.java | 12 ++++++++++-- .../integration/cases/WorkflowStartTestCase.java | 2 ++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinatorTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinatorTest.java index e2e96a96140a..8e838b724a21 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinatorTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/engine/task/dispatcher/WorkerGroupDispatcherCoordinatorTest.java @@ -24,25 +24,33 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.junit.jupiter.api.BeforeEach; 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; +import org.springframework.test.util.ReflectionTestUtils; @ExtendWith(MockitoExtension.class) class WorkerGroupDispatcherCoordinatorTest { - @InjectMocks private WorkerGroupDispatcherCoordinator workerGroupDispatcherCoordinator; @Mock private ITaskExecutorClient taskExecutorClient; + @BeforeEach + void setUp() { + MasterConfig masterConfig = new MasterConfig(); + workerGroupDispatcherCoordinator = new WorkerGroupDispatcherCoordinator(masterConfig); + ReflectionTestUtils.setField(workerGroupDispatcherCoordinator, "taskExecutorClient", taskExecutorClient); + } + @Test void addTaskToWorkerGroup_NewWorkerGroup_ShouldAddTask() { String workerGroup = "newGroup"; diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java index 3010a2b4d937..dffd6e6c240a 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java @@ -1452,6 +1452,7 @@ public void testStartWorkflow_with_workerGroupNotFoundAndTimeoutEnabled() { final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() .workflowDefinition(workflow) + .runWorkflowCommandParam(new RunWorkflowCommandParam()) .build(); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO); @@ -1494,6 +1495,7 @@ public void testTaskStaysRunning_whenWorkerGroupNotFoundAndTimeoutDisabled() { final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() .workflowDefinition(workflow) + .runWorkflowCommandParam(new RunWorkflowCommandParam()) .build(); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);