diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/IWorkflowExecuteRunnableRepository.java similarity index 73% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/IWorkflowExecuteRunnableRepository.java index 1352d125c377..a67017630164 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/IWorkflowExecuteRunnableRepository.java @@ -17,24 +17,17 @@ package org.apache.dolphinscheduler.server.master.cache; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; - import java.util.Collection; -import lombok.NonNull; - -/** - * cache of process instance id and WorkflowExecuteThread - */ -public interface ProcessInstanceExecCacheManager { +public interface IWorkflowExecuteRunnableRepository { /** - * get WorkflowExecuteThread by process instance id + * Get IWorkflowExecutionRunnable by workflowInstance id * - * @param processInstanceId processInstanceId - * @return WorkflowExecuteThread + * @param processInstanceId workflowInstanceId + * @return IWorkflowExecutionRunnable */ - WorkflowExecuteRunnable getByProcessInstanceId(int processInstanceId); + W getByProcessInstanceId(int processInstanceId); /** * judge the process instance does it exist @@ -57,14 +50,14 @@ public interface ProcessInstanceExecCacheManager { * @param processInstanceId processInstanceId * @param workflowExecuteThread if it is null, will not be cached */ - void cache(int processInstanceId, @NonNull WorkflowExecuteRunnable workflowExecuteThread); + void cache(int processInstanceId, W workflowExecuteThread); /** * get all WorkflowExecuteThread from cache * * @return all WorkflowExecuteThread in cache */ - Collection getAll(); + Collection getAll(); void clearCache(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImpl.java similarity index 70% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImpl.java index 105aa3970879..82a20ea9eb8c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImpl.java @@ -17,37 +17,33 @@ package org.apache.dolphinscheduler.server.master.cache.impl; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; import java.util.Collection; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import javax.annotation.PostConstruct; - import lombok.NonNull; import org.springframework.stereotype.Component; import com.google.common.collect.ImmutableList; -/** - * cache of process instance id and WorkflowExecuteThread - */ @Component -public class ProcessInstanceExecCacheManagerImpl implements ProcessInstanceExecCacheManager { +public class WorkflowExecuteRunnableRepositoryImpl + implements + IWorkflowExecuteRunnableRepository { - private final ConcurrentHashMap processInstanceExecMaps = - new ConcurrentHashMap<>(); + private final Map processInstanceExecMaps = new ConcurrentHashMap<>(); - @PostConstruct - public void registerMetrics() { + public WorkflowExecuteRunnableRepositoryImpl() { ProcessInstanceMetrics.registerProcessInstanceRunningGauge(processInstanceExecMaps::size); } @Override - public WorkflowExecuteRunnable getByProcessInstanceId(int processInstanceId) { + public IWorkflowExecutionRunnable getByProcessInstanceId(int processInstanceId) { return processInstanceExecMaps.get(processInstanceId); } @@ -62,12 +58,12 @@ public void removeByProcessInstanceId(int processInstanceId) { } @Override - public void cache(int processInstanceId, @NonNull WorkflowExecuteRunnable workflowExecuteThread) { + public void cache(int processInstanceId, @NonNull IWorkflowExecutionRunnable workflowExecuteThread) { processInstanceExecMaps.put(processInstanceId, workflowExecuteThread); } @Override - public Collection getAll() { + public Collection getAll() { return ImmutableList.copyOf(processInstanceExecMaps.values()); } 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 02c0dcb819d6..29233e447c48 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 @@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostSelector; import org.apache.dolphinscheduler.server.master.processor.queue.TaskExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.commons.lang3.StringUtils; @@ -59,7 +59,7 @@ public class MasterConfig implements Validator { /** * todo: We may need to split the process/task into different thread size. * The thread number used to handle processInstance and task event. - * Will create two thread poll to execute {@link WorkflowExecuteRunnable} and {@link TaskExecuteRunnable}. + * Will create two thread poll to execute {@link DefaultWorkflowExecutionRunnable} and {@link TaskExecuteRunnable}. */ private int execThreads = 10; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEventHandler.java index 08ee6b66c5e3..f64c0ebab71a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEventHandler.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.server.master.event; import org.apache.dolphinscheduler.common.enums.StateEventType; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; public interface StateEventHandler { @@ -30,7 +30,7 @@ public interface StateEventHandler { * @throws StateEventHandleError this exception means it cannot be recovered, so the event need to drop. * @throws StateEventHandleException this means it can be recovered. */ - boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleException, StateEventHandleError, StateEventHandleFailure; StateEventType getEventType(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandler.java index 158e66a8b603..5b5e138c451e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandler.java @@ -23,9 +23,9 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import org.apache.dolphinscheduler.server.master.utils.DataQualityResultOperator; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -40,7 +40,7 @@ public class TaskCacheEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -64,8 +64,9 @@ public void handleTaskEvent(TaskEvent taskEvent) { int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = processInstanceExecCacheManager.getByProcessInstanceId( - processInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId( + processInstanceId); Optional taskInstanceOptional = workflowExecuteRunnable.getTaskInstance(taskInstanceId); if (!taskInstanceOptional.isPresent()) { return; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java index f749ac46cd72..b6ce77f0de2e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java @@ -25,10 +25,10 @@ import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceExecutionEventAckListener; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionRunningEventAck; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import java.util.Optional; @@ -43,7 +43,7 @@ public class TaskDelayEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private TaskInstanceDao taskInstanceDao; @@ -59,8 +59,8 @@ public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = - this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + this.IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); if (workflowExecuteRunnable == null) { sendAckToWorker(taskEvent); throw new TaskEventHandleError("Cannot find related workflow instance from cache"); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java index 112aec4a1b5b..9db9b4d914c5 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java @@ -22,9 +22,9 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -36,7 +36,7 @@ public class TaskDispatchEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private TaskInstanceDao taskInstanceDao; @@ -46,13 +46,13 @@ public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = - this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + IWorkflowExecutionRunnable workflowExecuteRunnable = + this.IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); if (workflowExecuteRunnable == null) { throw new TaskEventHandleError("Cannot find related workflow instance from cache"); } - TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId) - .orElseThrow(() -> new TaskEventHandleError("Cannot find related taskInstance from cache")); + TaskInstance taskInstance = workflowExecuteRunnable.getTaskExecutionRunnableById(taskInstanceId) + .getTaskExecutionRunnableContext().getTaskInstance(); if (taskInstance.getState() != TaskExecutionStatus.SUBMITTED_SUCCESS) { log.warn( "The current taskInstance status is not SUBMITTED_SUCCESS, so the dispatch event will be discarded, the current is a delay event, event: {}", diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java index f3d3a7480adf..6e83274d2317 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java @@ -25,16 +25,14 @@ import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceExecutionEventAckListener; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionFinishEventAck; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import org.apache.dolphinscheduler.server.master.utils.DataQualityResultOperator; import org.apache.dolphinscheduler.service.process.ProcessService; -import java.util.Optional; - import lombok.extern.slf4j.Slf4j; import org.springframework.beans.factory.annotation.Autowired; @@ -45,7 +43,7 @@ public class TaskResultEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -67,20 +65,21 @@ public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError, Ta int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId( - processInstanceId); + IWorkflowExecutionRunnable workflowExecuteRunnable = + this.IWorkflowExecuteRunnableRepository.getByProcessInstanceId( + processInstanceId); if (workflowExecuteRunnable == null) { sendAckToWorker(taskEvent); throw new TaskEventHandleError( "Handle task result event error, cannot find related workflow instance from cache, will discard this event"); } - Optional taskInstanceOptional = workflowExecuteRunnable.getTaskInstance(taskInstanceId); - if (!taskInstanceOptional.isPresent()) { + TaskInstance taskInstance = workflowExecuteRunnable.getTaskExecutionRunnableById(taskInstanceId) + .getTaskExecutionRunnableContext().getTaskInstance(); + if (taskInstance == null) { sendAckToWorker(taskEvent); throw new TaskEventHandleError( "Handle task result event error, cannot find the taskInstance from cache, will discord this event"); } - TaskInstance taskInstance = taskInstanceOptional.get(); if (taskInstance.getState().isFinished()) { sendAckToWorker(taskEvent); throw new TaskEventHandleError( diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java index 074be181029f..c203ed052b98 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.enums.StateEventType; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import java.util.Map; @@ -33,7 +33,7 @@ public class TaskRetryStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleException { TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java index 14cb8571d9d9..0ca8871f1177 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java @@ -25,10 +25,10 @@ import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceExecutionEventAckListener; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionRunningEventAck; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import java.util.Optional; @@ -40,7 +40,7 @@ public class TaskRunningEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -56,8 +56,8 @@ public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = - this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // this.IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); if (workflowExecuteRunnable == null) { sendAckToWorker(taskEvent); throw new TaskEventHandleError( diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java index 2dde1315de07..3a9d8e244044 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.enums.StateEventType; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import java.util.Optional; import java.util.Set; @@ -34,7 +34,7 @@ public class TaskStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleException, StateEventHandleError { TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent; measureTaskState(taskStateEvent); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java index c04eb9c338bb..16cbe1895857 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java @@ -22,8 +22,8 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import java.util.Map; @@ -36,7 +36,7 @@ public class TaskTimeoutStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleError, StateEventHandleException { TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent; @@ -56,7 +56,7 @@ public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, } TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine() .getTimeoutNotifyStrategy(); - Map taskExecuteRunnableMap = + Map taskExecuteRunnableMap = workflowExecuteRunnable.getTaskExecuteRunnableMap(); if ((TaskTimeoutStrategy.FAILED == taskTimeoutStrategy || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy)) { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java index 5be19ddfdbf7..8187e12e43d2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java @@ -24,9 +24,9 @@ import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceExecutionEventAckListener; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionInfoEventAck; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import java.util.Optional; @@ -38,7 +38,7 @@ public class TaskUpdatePidEventHandler implements TaskEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -51,8 +51,8 @@ public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { int taskInstanceId = taskEvent.getTaskInstanceId(); int processInstanceId = taskEvent.getProcessInstanceId(); - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); if (workflowExecuteRunnable == null) { sendAckToWorker(taskEvent); throw new TaskEventHandleError( diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowBlockStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowBlockStateEventHandler.java index 07f89fc5445f..f1c40a15c638 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowBlockStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowBlockStateEventHandler.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import java.util.Optional; @@ -34,7 +34,7 @@ public class WorkflowBlockStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleError { log.info("Handle workflow instance state block event"); Optional taskInstanceOptional = diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java index c7d4032565c4..6269cf07758f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java @@ -17,17 +17,9 @@ package org.apache.dolphinscheduler.server.master.event; -import org.apache.dolphinscheduler.common.enums.StateEventType; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; -import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.runner.StateWheelExecuteThread; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; -import org.apache.dolphinscheduler.server.master.runner.WorkflowStartStatus; - -import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; @@ -39,7 +31,7 @@ public class WorkflowStartEventHandler implements WorkflowEventHandler { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private StateWheelExecuteThread stateWheelExecuteThread; @@ -49,36 +41,35 @@ public class WorkflowStartEventHandler implements WorkflowEventHandler { @Override public void handleWorkflowEvent(final WorkflowEvent workflowEvent) throws WorkflowEventHandleError { - log.info("Handle workflow start event, begin to start a workflow, event: {}", workflowEvent); - WorkflowExecuteRunnable workflowExecuteRunnable = processInstanceExecCacheManager.getByProcessInstanceId( - workflowEvent.getWorkflowInstanceId()); - if (workflowExecuteRunnable == null) { - throw new WorkflowEventHandleError( - "The workflow start event is invalid, cannot find the workflow instance from cache"); - } - ProcessInstance processInstance = - workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(); - ProcessInstanceMetrics.incProcessInstanceByStateAndProcessDefinitionCode("submit", - processInstance.getProcessDefinitionCode().toString()); - CompletableFuture.supplyAsync(workflowExecuteRunnable::call, workflowExecuteThreadPool) - .thenAccept(workflowStartStatus -> { - if (WorkflowStartStatus.SUCCESS == workflowStartStatus) { - log.info("Success submit the workflow instance"); - if (processInstance.getTimeout() > 0) { - stateWheelExecuteThread.addProcess4TimeoutCheck(processInstance); - } - } else if (WorkflowStartStatus.FAILED == workflowStartStatus) { - log.error( - "Failed to submit the workflow instance, will send fail state event: {}", - workflowEvent); - WorkflowStateEvent stateEvent = WorkflowStateEvent.builder() - .processInstanceId(processInstance.getId()) - .type(StateEventType.PROCESS_SUBMIT_FAILED) - .status(WorkflowExecutionStatus.FAILURE) - .build(); - workflowExecuteRunnable.addStateEvent(stateEvent); - } - }); + // log.info("Handle workflow start event, begin to start a workflow, event: {}", workflowEvent); + // DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId( + // workflowEvent.getWorkflowInstanceId()); + // if (workflowExecuteRunnable == null) { + // throw new WorkflowEventHandleError( + // "The workflow start event is invalid, cannot find the workflow instance from cache"); + // } + // ProcessInstance processInstance = + // workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(); + // ProcessInstanceMetrics.incProcessInstanceByStateAndProcessDefinitionCode("submit", + // processInstance.getProcessDefinitionCode().toString()); + // CompletableFuture.runAsync(workflowExecuteRunnable::start, workflowExecuteThreadPool) + // .thenAccept((unused) -> { + // log.info("Success submit the workflow instance"); + // if (processInstance.getTimeout() > 0) { + // stateWheelExecuteThread.addProcess4TimeoutCheck(processInstance); + // } + // }) + // .exceptionally(e -> { + // log.error( + // "Failed to submit the workflow instance, will send fail state event: {}", + // .processInstanceId(processInstance.getId()) + // .type(StateEventType.PROCESS_SUBMIT_FAILED) + // .status(WorkflowExecutionStatus.FAILURE) + // .build(); + // workflowExecuteRunnable.addStateEvent(stateEvent); + // return null; + // }); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java index 018cff565ae4..1a694a881fcb 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -32,7 +32,7 @@ public class WorkflowStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleException { WorkflowStateEvent workflowStateEvent = (WorkflowStateEvent) stateEvent; ProcessInstance processInstance = diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowSubmitFailStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowSubmitFailStateEventHandler.java index c4d21ed2309a..0e782e64666b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowSubmitFailStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowSubmitFailStateEventHandler.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.enums.StateEventType; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -31,7 +31,7 @@ public class WorkflowSubmitFailStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) throws StateEventHandleException { WorkflowStateEvent workflowStateEvent = (WorkflowStateEvent) stateEvent; ProcessInstance processInstance = diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowTimeoutStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowTimeoutStateEventHandler.java index c4d0555c1caa..ab3b14c6699b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowTimeoutStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowTimeoutStateEventHandler.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.enums.StateEventType; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -31,7 +31,7 @@ public class WorkflowTimeoutStateEventHandler implements StateEventHandler { @Override - public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent) { + public boolean handleStateEvent(DefaultWorkflowExecutionRunnable workflowExecuteRunnable, StateEvent stateEvent) { log.info("Handle workflow instance timeout event"); ProcessInstance processInstance = workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventDispatcher.java new file mode 100644 index 000000000000..3aab1aef3e5a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventDispatcher.java @@ -0,0 +1,53 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class EventDispatcher implements IEventDispatcher { + + @Autowired + private EventEngine eventEngine; + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void start() { + log.info(getClass().getName() + " started"); + } + + @Override + public void dispatchEvent(IEvent iEvent) { + Integer workflowInstanceId; + if (iEvent instanceof IWorkflowExecutionRunnableEvent) { + workflowInstanceId = ((IWorkflowExecutionRunnableEvent) iEvent).getWorkflowInstanceId(); + } else if (iEvent instanceof ITaskExecutionRunnableEvent) { + workflowInstanceId = ((ITaskExecutionRunnableEvent) iEvent).getWorkflowInstanceId(); + } else { + throw new IllegalArgumentException("Unsupported event type: " + iEvent.getClass().getName()); + } + + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + workflowExecuteRunnable.getEventRepository().storeEventToTail(iEvent); + log.debug("Success dispatch event {} to EventRepository", iEvent); + eventEngine.notify(); + } + + @Override + public void stop() { + log.info(getClass().getName() + " stopped"); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java new file mode 100644 index 000000000000..d7eb07c234f3 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java @@ -0,0 +1,90 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import org.apache.commons.lang3.time.StopWatch; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Component +public class EventEngine extends BaseDaemonThread { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private EventFirer eventFirer; + + private final Set firingWorkflowInstanceIds = ConcurrentHashMap.newKeySet(); + + public EventEngine() { + super("EventEngine"); + } + + @Override + public synchronized void start() { + super.start(); + log.info(getClass().getName() + " started"); + } + + @Override + public void run() { + for (;;) { + try { + StopWatch stopWatch = StopWatch.createStarted(); + fireAllActiveEvents(); + stopWatch.stop(); + log.info("Fire all active events cost: {} ms", stopWatch.getTime()); + this.wait(5_000); + } catch (Throwable throwable) { + log.error("Fire active event error", throwable); + ThreadUtils.sleep(3_000); + } + } + } + + public void fireAllActiveEvents() { + Collection workflowExecutionRunnableCollection = + workflowExecuteRunnableRepository.getAll(); + for (IWorkflowExecutionRunnable workflowExecutionRunnable : workflowExecutionRunnableCollection) { + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + final Integer workflowInstanceId = workflowInstance.getId(); + final String workflowInstanceName = workflowInstance.getName(); + try { + LogUtils.setWorkflowInstanceIdMDC(workflowInstanceId); + if (firingWorkflowInstanceIds.contains(workflowInstanceId)) { + log.debug("WorkflowExecutionRunnable: {} is already in firing", workflowInstanceName); + return; + } + IEventRepository workflowEventRepository = workflowExecutionRunnable.getEventRepository(); + firingWorkflowInstanceIds.add(workflowInstanceId); + eventFirer.fireActiveEvents(workflowEventRepository) + .whenComplete((fireCount, ex) -> { + firingWorkflowInstanceIds.remove(workflowInstanceId); + if (ex != null) { + log.error("Fire event for WorkflowExecutionRunnable: {} error", workflowInstanceName, ex); + } else { + log.info("Fire {} events for WorkflowExecutionRunnable: {} success", fireCount, workflowInstanceName); + } + }); + } finally { + LogUtils.removeWorkflowInstanceIdMDC(); + } + } + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFireThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFireThreadPool.java new file mode 100644 index 000000000000..adaa8b33d990 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFireThreadPool.java @@ -0,0 +1,40 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.function.Supplier; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class EventFireThreadPool { + + private final MasterConfig masterConfig; + + private final ThreadPoolExecutor threadPoolExecutor; + + public EventFireThreadPool(MasterConfig masterConfig) { + this.masterConfig = masterConfig; + this.threadPoolExecutor = + ThreadUtils.newDaemonFixedThreadExecutor(getClass().getName(), masterConfig.getExecThreads()); + } + + public ThreadPoolExecutor getThreadPoolExecutor() { + return threadPoolExecutor; + } + + public CompletableFuture submitEventFireTask(Supplier supplier) { + return CompletableFuture.supplyAsync(supplier, threadPoolExecutor); + } + + public CompletableFuture submitEventFireTask(Runnable runnable) { + return CompletableFuture.runAsync(runnable, threadPoolExecutor); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java new file mode 100644 index 000000000000..5a3e99ca5acd --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java @@ -0,0 +1,76 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; + +import java.util.concurrent.CompletableFuture; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class EventFirer implements IEventFirer { + + protected final IEventOperatorManager eventOperatorManager; + + protected final EventFireThreadPool eventFireThreadPool; + + public EventFirer(IEventOperatorManager eventOperatorManager, + EventFireThreadPool eventFireThreadPool) { + this.eventOperatorManager = eventOperatorManager; + this.eventFireThreadPool = eventFireThreadPool; + } + + @Override + public CompletableFuture fireActiveEvents(IEventRepository eventRepository) { + if (eventRepository.getEventSize() == 0) { + return CompletableFuture.completedFuture(0); + } + return CompletableFuture.supplyAsync(() -> { + int fireCount = 0; + for (;;) { + IEvent event = eventRepository.poolEvent(); + if (event == null) { + break; + } + + if (event instanceof IAsyncEvent) { + fireAsyncEvent(event); + fireCount++; + continue; + } + try { + fireSyncEvent(event); + fireCount++; + } catch (Exception ex) { + if (ExceptionUtils.isDatabaseConnectedFailedException(ex)) { + // If the event is failed due to cannot connect to DB, we should retry it + eventRepository.storeEventToHead(event); + } + throw ex; + } + } + return fireCount; + }, eventFireThreadPool.getThreadPoolExecutor()); + } + + private void fireAsyncEvent(IEvent event) { + CompletableFuture.runAsync(() -> { + log.info("Begin fire IAsyncEvent: {}", event); + eventOperatorManager.getEventOperator(event).handleEvent(event); + log.info("Success fire IAsyncEvent: {}", event); + }, eventFireThreadPool.getThreadPoolExecutor()) + .exceptionally(ex -> { + log.error("Failed to fire IAsyncEvent: {}", event, ex); + return null; + }); + } + + private void fireSyncEvent(IEvent event) { + log.info("Begin fire SyncEvent: {}", event); + eventOperatorManager.getEventOperator(event).handleEvent(event); + log.info("Success fire SyncEvent: {}", event); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java new file mode 100644 index 000000000000..0348919c668d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java @@ -0,0 +1,19 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +/** + * The event operator manager interface used to get {@link ITaskExecutionRunnableEventOperator}. + */ +@Slf4j +@Component +public class EventOperatorManager implements IEventOperatorManager { + + @Override + public IEventOperator getEventOperator(IEvent event) { + return null; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventRepository.java new file mode 100644 index 000000000000..972788b67f24 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventRepository.java @@ -0,0 +1,36 @@ +package org.apache.dolphinscheduler.server.master.events; + +import java.util.concurrent.LinkedBlockingDeque; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class EventRepository implements IEventRepository { + + protected final LinkedBlockingDeque eventQueue; + + public EventRepository() { + this.eventQueue = new LinkedBlockingDeque<>(); + } + + @Override + public void storeEventToTail(E event) { + eventQueue.offerLast(event); + } + + @Override + public void storeEventToHead(E event) { + eventQueue.offerFirst(event); + } + + @Override + public E poolEvent() { + return eventQueue.poll(); + } + + @Override + public int getEventSize() { + return eventQueue.size(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java new file mode 100644 index 000000000000..1ea455355494 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java @@ -0,0 +1,7 @@ +package org.apache.dolphinscheduler.server.master.events; + +/** + * Mark the event as AsyncEvent, if the event is marked as AsyncEvent, the event will be handled asynchronously and we don't . + */ +public interface IAsyncEvent { +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java new file mode 100644 index 000000000000..425ba97d49a2 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java @@ -0,0 +1,5 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface IEvent { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventDispatcher.java new file mode 100644 index 000000000000..e32f8bc619e8 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventDispatcher.java @@ -0,0 +1,15 @@ +package org.apache.dolphinscheduler.server.master.events; + +/** + * The event dispatcher interface used to dispatch event. + * Each event should be dispatched to the corresponding workflow event queue. + */ +public interface IEventDispatcher { + + void start(); + + void stop(); + + void dispatchEvent(E event); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java new file mode 100644 index 000000000000..576942089db2 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java @@ -0,0 +1,19 @@ +package org.apache.dolphinscheduler.server.master.events; + +import java.util.concurrent.CompletableFuture; + +/** + * The event firer interface used to fire event. + * + * @param event type + */ +public interface IEventFirer { + + /** + * Fire all active events in the event repository + * + * @return the count of fired success events + */ + CompletableFuture fireActiveEvents(IEventRepository eventRepository); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java new file mode 100644 index 000000000000..ef767e35d011 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java @@ -0,0 +1,15 @@ +package org.apache.dolphinscheduler.server.master.events; + +/** + * The event operator interface used to handle event. + */ +public interface IEventOperator { + + /** + * Handle the given event + * + * @param event event + */ + void handleEvent(E event); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java new file mode 100644 index 000000000000..0670dc6d57eb --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java @@ -0,0 +1,16 @@ +package org.apache.dolphinscheduler.server.master.events; + +/** + * The event operator manager interface used to get event operator. + */ +public interface IEventOperatorManager { + + /** + * Get the {@link IEventOperator} for the given event. + * + * @param event event + * @return event operator for the given event + */ + IEventOperator getEventOperator(E event); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java new file mode 100644 index 000000000000..b6f3d284f533 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java @@ -0,0 +1,16 @@ +package org.apache.dolphinscheduler.server.master.events; + +/** + * The event repository interface used to store event. + */ +public interface IEventRepository { + + void storeEventToTail(E event); + + void storeEventToHead(E event); + + E poolEvent(); + + int getEventSize(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java new file mode 100644 index 000000000000..95de44d0b847 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java @@ -0,0 +1,4 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface ISyncEvent { +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEvent.java new file mode 100644 index 000000000000..5c37f55f1c9b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEvent.java @@ -0,0 +1,9 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface ITaskExecutionRunnableEvent extends IEvent { + + Integer getWorkflowInstanceId(); + + Integer getTaskInstanceId(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEventOperator.java new file mode 100644 index 000000000000..2437e32db325 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskExecutionRunnableEventOperator.java @@ -0,0 +1,5 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface ITaskExecutionRunnableEventOperator extends IEventOperator { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEvent.java new file mode 100644 index 000000000000..399ad7e69575 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEvent.java @@ -0,0 +1,7 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface IWorkflowExecutionRunnableEvent extends IEvent { + + Integer getWorkflowInstanceId(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEventOperator.java new file mode 100644 index 000000000000..af59c4691b68 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowExecutionRunnableEventOperator.java @@ -0,0 +1,6 @@ +package org.apache.dolphinscheduler.server.master.events; + +public interface IWorkflowExecutionRunnableEventOperator + extends + IEventOperator { +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEvent.java new file mode 100644 index 000000000000..f05d4842f0d8 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEvent.java @@ -0,0 +1,18 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowExecutionRunnableFailedEvent implements IWorkflowExecutionRunnableEvent, ISyncEvent { + + private Integer workflowInstanceId; + + private String failedReason; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEventOperator.java new file mode 100644 index 000000000000..890729d998da --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFailedEventOperator.java @@ -0,0 +1,44 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableFailedEventOperator + implements + IWorkflowExecutionRunnableEventOperator { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private EventDispatcher eventDispatcher; + + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Override + public void handleEvent(WorkflowExecutionRunnableFailedEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + workflowInstance.setState(WorkflowExecutionStatus.FAILURE); + processInstanceDao.updateById(workflowInstance); + log.info("Handle WorkflowExecutionRunnableFailedEvent success, set workflowInstance status to {}", + workflowInstance.getState()); + + eventDispatcher.dispatchEvent(new WorkflowExecutionRunnableFinalizeEvent(workflowInstanceId)); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEvent.java new file mode 100644 index 000000000000..327e3668eb35 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEvent.java @@ -0,0 +1,15 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@AllArgsConstructor +@NoArgsConstructor +public class WorkflowExecutionRunnableFinalizeEvent implements IWorkflowExecutionRunnableEvent, ISyncEvent { + + private Integer workflowInstanceId; +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEventOperator.java new file mode 100644 index 000000000000..52259cd5de11 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableFinalizeEventOperator.java @@ -0,0 +1,26 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableFinalizeEventOperator + implements + IWorkflowExecutionRunnableEventOperator { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowExecutionRunnableFinalizeEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + workflowExecuteRunnableRepository.removeByProcessInstanceId(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEvent.java new file mode 100644 index 000000000000..5f9b5dfe3f7b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEvent.java @@ -0,0 +1,20 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowExecutionRunnableTriggerEvent implements IWorkflowExecutionRunnableEvent, ISyncEvent { + + private Integer workflowInstanceId; + + @Override + public Integer getWorkflowInstanceId() { + return workflowInstanceId; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEventOperator.java new file mode 100644 index 000000000000..06429b636570 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerEventOperator.java @@ -0,0 +1,45 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableTriggerEventOperator + implements + IWorkflowExecutionRunnableEventOperator { + + @Autowired + private EventDispatcher eventDispatcher; + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowExecutionRunnableTriggerEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + try { + workflowExecutionRunnable.start(); + } catch (Throwable exception) { + if (ExceptionUtils.isDatabaseConnectedFailedException(exception)) { + throw exception; + } + log.error("Handle : {} failed", event, exception); + WorkflowExecutionRunnableFailedEvent workflowExecutionRunnableFailedEvent = + WorkflowExecutionRunnableFailedEvent.builder() + .workflowInstanceId(workflowInstanceId) + .failedReason(exception.getMessage()) + .build(); + eventDispatcher.dispatchEvent(workflowExecutionRunnableFailedEvent); + } + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEvent.java new file mode 100644 index 000000000000..f4aa7aeb386a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEvent.java @@ -0,0 +1,25 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowExecutionRunnableTriggerNextTaskEvent implements IWorkflowExecutionRunnableEvent, ISyncEvent { + + private int workflowInstanceId; + + /** + * The task code of the parent task, if it is the root task, the value is null + */ + private Long parentTaskCode; + + @Override + public Integer getWorkflowInstanceId() { + return workflowInstanceId; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEventOperator.java new file mode 100644 index 000000000000..73e96bd64bc6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggerNextTaskEventOperator.java @@ -0,0 +1,27 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableTriggerNextTaskEventOperator + implements + IWorkflowExecutionRunnableEventOperator { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowExecutionRunnableTriggerNextTaskEvent event) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(event.getWorkflowInstanceId()); + workflowExecutionRunnable.triggerNextTasks(event.getParentTaskCode()); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEvent.java new file mode 100644 index 000000000000..8b80366ec363 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEvent.java @@ -0,0 +1,16 @@ +package org.apache.dolphinscheduler.server.master.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowExecutionRunnableTriggeredEvent implements IWorkflowExecutionRunnableEvent, IAsyncEvent { + + private int workflowInstanceId; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEventOperator.java new file mode 100644 index 000000000000..e463be1a8112 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowExecutionRunnableTriggeredEventOperator.java @@ -0,0 +1,45 @@ +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProjectUser; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager; +import org.apache.dolphinscheduler.service.process.ProcessService; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableTriggeredEventOperator + implements + IWorkflowExecutionRunnableEventOperator { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private ProcessService processService; + + @Autowired + private ListenerEventAlertManager listenerEventAlertManager; + + @Override + public void handleEvent(WorkflowExecutionRunnableTriggeredEvent event) { + int workflowInstanceId = event.getWorkflowInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + Long workflowDefinitionCode = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowDefinition().getCode(); + ProcessInstanceMetrics.incProcessInstanceByStateAndProcessDefinitionCode("submit", workflowDefinitionCode); + + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(workflowInstance.getId()); + listenerEventAlertManager.publishProcessStartListenerEvent(workflowInstance, projectUser); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java new file mode 100644 index 000000000000..19a11b9639bb --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java @@ -0,0 +1,12 @@ +package org.apache.dolphinscheduler.server.master.exception; + +public class TaskExecuteRunnableNotFoundException extends RuntimeException { + + public TaskExecuteRunnableNotFoundException(Integer workflowInstanceId) { + super("WorkflowExecuteRunnable not found: [id=" + workflowInstanceId + "]"); + } + + public TaskExecuteRunnableNotFoundException(String workflowInstanceName) { + super("WorkflowExecuteRunnable not found: [name=" + workflowInstanceName + "]"); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java deleted file mode 100644 index ac37c94438aa..000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java +++ /dev/null @@ -1,26 +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.exception; - -public class TaskExecutionContextCreateException extends MasterException { - - public TaskExecutionContextCreateException(String message) { - super(message); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java new file mode 100644 index 000000000000..3ded58f3c6b1 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java @@ -0,0 +1,13 @@ +package org.apache.dolphinscheduler.server.master.exception; + +public class WorkflowExecuteRunnableNotFoundException extends RuntimeException { + + public WorkflowExecuteRunnableNotFoundException(Integer workflowInstanceId) { + super("WorkflowExecuteRunnable not found: [id=" + workflowInstanceId + "]"); + } + + public WorkflowExecuteRunnableNotFoundException(String workflowInstanceName) { + super("WorkflowExecuteRunnable not found: [name=" + workflowInstanceName + "]"); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java index 2db3d04c0b7c..4f7ac95f0307 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java @@ -90,6 +90,18 @@ public void incProcessInstanceByStateAndProcessDefinitionCode(final String state .increment(); } + public void incProcessInstanceByStateAndProcessDefinitionCode(final String state, + final Long processDefinitionCode) { + // When tags need to be determined from local context, + // you have no choice but to construct or lookup the Meter inside your method body. + // The lookup cost is just a single hash lookup, so it is acceptable for most use cases. + Metrics.globalRegistry.counter( + "ds.workflow.instance.count", + "state", state, + "process.definition.code", processDefinitionCode.toString()) + .increment(); + } + public void cleanUpProcessInstanceCountMetricsByDefinitionCode(final Long processDefinitionCode) { for (final String state : processInstanceStates) { final Counter counter = Metrics.globalRegistry.counter( diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java index 03f254a09c76..c9b34b5002f0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java @@ -20,9 +20,9 @@ import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.event.StateEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import java.util.ArrayList; @@ -53,7 +53,7 @@ public class StateEventResponseService { private Thread responseWorker; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -130,14 +130,14 @@ public void run() { private void persist(StateEvent stateEvent) { try { - if (!this.processInstanceExecCacheManager.contains(stateEvent.getProcessInstanceId())) { + if (!this.IWorkflowExecuteRunnableRepository.contains(stateEvent.getProcessInstanceId())) { log.warn("Persist event into workflow execute thread error, " + "cannot find the workflow instance from cache manager, event: {}", stateEvent); return; } - WorkflowExecuteRunnable workflowExecuteThread = - this.processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId()); + DefaultWorkflowExecutionRunnable workflowExecuteThread = + this.IWorkflowExecuteRunnableRepository.getByProcessInstanceId(stateEvent.getProcessInstanceId()); // We will refresh the task instance status first, if the refresh failed the event will not be removed switch (stateEvent.getType()) { case TASK_STATE_CHANGE: diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteRunnable.java index 78199f517d54..39b62d57ae81 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteRunnable.java @@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.server.master.event.TaskEventHandleError; import org.apache.dolphinscheduler.server.master.event.TaskEventHandleException; import org.apache.dolphinscheduler.server.master.event.TaskEventHandler; +import org.apache.dolphinscheduler.server.master.events.ITaskExecutionRunnableEvent; +import org.apache.dolphinscheduler.server.master.runner.IEventfulExecutionRunnable; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; @@ -32,7 +34,7 @@ * task execute thread */ @Slf4j -public class TaskExecuteRunnable implements Runnable { +public class TaskExecuteRunnable implements Runnable, IEventfulExecutionRunnable { private final int processInstanceId; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java index 0bdfbb0e3e12..bf8fb8f30871 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.server.master.processor.queue; import org.apache.dolphinscheduler.common.enums.TaskEventType; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.TaskEventHandler; @@ -48,7 +48,7 @@ public class TaskExecuteThreadPool extends ThreadPoolTaskExecutor { private MasterConfig masterConfig; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private List taskEventHandlerList; @@ -81,7 +81,7 @@ public void submitTaskEvent(TaskEvent taskEvent) { .addTaskEvent(taskEvent); return; } - if (!processInstanceExecCacheManager.contains(taskEvent.getProcessInstanceId())) { + if (!IWorkflowExecuteRunnableRepository.contains(taskEvent.getProcessInstanceId())) { log.warn("Cannot find workflowExecuteThread from cacheManager, event: {}", taskEvent); return; } @@ -111,7 +111,7 @@ public void executeEvent(TaskExecuteRunnable taskExecuteThread) { public void onFailure(Throwable ex) { Integer processInstanceId = taskExecuteThread.getProcessInstanceId(); log.error("[WorkflowInstance-{}] persist event failed", processInstanceId, ex); - if (!processInstanceExecCacheManager.contains(processInstanceId)) { + if (!IWorkflowExecuteRunnableRepository.contains(processInstanceId)) { taskExecuteThreadMap.remove(processInstanceId); log.info( "[WorkflowInstance-{}] Cannot find processInstance from cacheManager, remove process instance from threadMap", @@ -124,7 +124,7 @@ public void onFailure(Throwable ex) { public void onSuccess(Object result) { Integer processInstanceId = taskExecuteThread.getProcessInstanceId(); log.info("[WorkflowInstance-{}] persist events succeeded", processInstanceId); - if (!processInstanceExecCacheManager.contains(processInstanceId)) { + if (!IWorkflowExecuteRunnableRepository.contains(processInstanceId)) { taskExecuteThreadMap.remove(processInstanceId); log.info( "[WorkflowInstance-{}] Cannot find processInstance from cacheManager, remove process instance from threadMap", diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterWaitingStrategy.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterWaitingStrategy.java index 2647542a18ae..40345ded086e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterWaitingStrategy.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterWaitingStrategy.java @@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.RegistryException; import org.apache.dolphinscheduler.registry.api.StrategyType; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.WorkflowEventQueue; import org.apache.dolphinscheduler.server.master.runner.StateWheelExecuteThread; @@ -52,7 +52,7 @@ public class MasterWaitingStrategy implements MasterConnectStrategy { @Autowired private WorkflowEventQueue workflowEventQueue; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private StateWheelExecuteThread stateWheelExecuteThread; @@ -115,7 +115,7 @@ public StrategyType getStrategyType() { private void clearMasterResource() { workflowEventQueue.clearWorkflowEventQueue(); log.warn("Master clear workflow event queue due to lost registry connection"); - processInstanceExecCacheManager.clearCache(); + IWorkflowExecuteRunnableRepository.clearCache(); log.warn("Master clear process instance cache due to lost registry connection"); stateWheelExecuteThread.clearAllTasks(); log.warn("Master clear all state wheel task due to lost registry connection"); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskInstanceWakeupOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskInstanceWakeupOperationFunction.java index 4a83874fb961..21bcb5bed63a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskInstanceWakeupOperationFunction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskInstanceWakeupOperationFunction.java @@ -20,9 +20,9 @@ import org.apache.dolphinscheduler.extract.master.transportor.TaskInstanceWakeupRequest; import org.apache.dolphinscheduler.extract.master.transportor.TaskInstanceWakeupResponse; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -36,7 +36,7 @@ public class TaskInstanceWakeupOperationFunction ITaskInstanceOperationFunction { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Override public TaskInstanceWakeupResponse operate(TaskInstanceWakeupRequest taskInstanceWakeupRequest) { @@ -46,13 +46,13 @@ public TaskInstanceWakeupResponse operate(TaskInstanceWakeupRequest taskInstance int workflowInstanceId = taskInstanceWakeupRequest.getProcessInstanceId(); int taskInstanceId = taskInstanceWakeupRequest.getTaskInstanceId(); LogUtils.setWorkflowAndTaskInstanceIDMDC(workflowInstanceId, taskInstanceId); - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(workflowInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); if (workflowExecuteRunnable == null) { log.warn("cannot find WorkflowExecuteRunnable: {}, no need to Wakeup task", workflowInstanceId); return TaskInstanceWakeupResponse.failed("cannot find WorkflowExecuteRunnable: " + workflowInstanceId); } - DefaultTaskExecuteRunnable defaultTaskExecuteRunnable = + DefaultTaskExecutionRunnable defaultTaskExecuteRunnable = workflowExecuteRunnable.getTaskExecuteRunnableById(taskInstanceId).orElse(null); if (defaultTaskExecuteRunnable == null) { log.warn("Cannot find DefaultTaskExecuteRunnable: {}, cannot Wakeup task", taskInstanceId); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseITaskExecutionRunnable.java similarity index 61% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteRunnable.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseITaskExecutionRunnable.java index 10b456e51716..aac824c936c8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseITaskExecutionRunnable.java @@ -17,16 +17,19 @@ package org.apache.dolphinscheduler.server.master.runner; -import java.util.concurrent.Callable; +import static com.google.common.base.Preconditions.checkNotNull; -public interface IWorkflowExecuteRunnable extends Callable { - // todo: add control method to manage the workflow runnable e.g. pause/stop .... +public abstract class BaseITaskExecutionRunnable implements ITaskExecutionRunnable { - @Override - default WorkflowStartStatus call() { - return startWorkflow(); + protected final TaskExecutionRunnableContext taskExecutionRunnableContext; + + public BaseITaskExecutionRunnable(TaskExecutionRunnableContext taskExecutionRunnableContext) { + this.taskExecutionRunnableContext = checkNotNull(taskExecutionRunnableContext); } - WorkflowStartStatus startWorkflow(); + @Override + public TaskExecutionRunnableContext getTaskExecutionRunnableContext() { + return taskExecutionRunnableContext; + } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java index 30ab8fadec19..ed2843eb9a34 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java @@ -49,31 +49,36 @@ protected BaseTaskDispatcher(TaskEventService taskEventService, } @Override - public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { + public void dispatchTask(ITaskExecutionRunnable iTaskExecutionRunnable) throws TaskDispatchException { Host taskInstanceDispatchHost; try { - taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable) + taskInstanceDispatchHost = getTaskInstanceDispatchHost(iTaskExecutionRunnable) .orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task.")); } catch (WorkerGroupNotFoundException workerGroupNotFoundException) { log.error("Dispatch task: {} failed, worker group not found.", - taskExecuteRunnable.getTaskExecutionContext().getTaskName(), workerGroupNotFoundException); - addDispatchFailedEvent(taskExecuteRunnable); + iTaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext().getTaskName(), + workerGroupNotFoundException); + addDispatchFailedEvent(iTaskExecutionRunnable); return; } - taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress()); - doDispatch(taskExecuteRunnable); - taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress()); - log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(), + iTaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext() + .setHost(taskInstanceDispatchHost.getAddress()); + doDispatch(iTaskExecutionRunnable); + iTaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskInstance() + .setHost(taskInstanceDispatchHost.getAddress()); + log.info("Success dispatch task {} to {}.", + iTaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext().getTaskName(), taskInstanceDispatchHost.getAddress()); - addDispatchEvent(taskExecuteRunnable); + addDispatchEvent(iTaskExecutionRunnable); } - protected abstract void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException; + protected abstract void doDispatch(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException; - protected abstract Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; + protected abstract Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; - protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) { - TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + protected void addDispatchEvent(ITaskExecutionRunnable ITaskExecutionRunnable) { + TaskExecutionContext taskExecutionContext = + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext(); TaskEvent taskEvent = TaskEvent.newDispatchEvent( taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId(), @@ -81,8 +86,9 @@ protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) { taskEventService.addEvent(taskEvent); } - private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) { - TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + private void addDispatchFailedEvent(ITaskExecutionRunnable ITaskExecutionRunnable) { + TaskExecutionContext taskExecutionContext = + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext(); TaskEvent taskEvent = TaskEvent.builder() .processInstanceId(taskExecutionContext.getProcessInstanceId()) .taskInstanceId(taskExecutionContext.getTaskInstanceId()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskExecuteRunnable.java deleted file mode 100644 index fefdbf349384..000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskExecuteRunnable.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner; - -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; - -public abstract class BaseTaskExecuteRunnable implements TaskExecuteRunnable { - - protected final ProcessInstance workflowInstance; - protected final TaskInstance taskInstance; - protected final TaskExecutionContext taskExecutionContext; - - public BaseTaskExecuteRunnable(ProcessInstance workflowInstance, - TaskInstance taskInstance, - TaskExecutionContext taskExecutionContext) { - this.taskInstance = checkNotNull(taskInstance); - this.workflowInstance = checkNotNull(workflowInstance); - this.taskExecutionContext = checkNotNull(taskExecutionContext); - } - - @Override - public ProcessInstance getWorkflowInstance() { - return workflowInstance; - } - - @Override - public TaskInstance getTaskInstance() { - return taskInstance; - } - - @Override - public TaskExecutionContext getTaskExecutionContext() { - return taskExecutionContext; - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecutionRunnable.java similarity index 72% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecuteRunnable.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecutionRunnable.java index c1b13717bdd3..44ef4685c25d 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecutionRunnable.java @@ -19,20 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.server.master.runner.operator.TaskExecuteRunnableOperatorManager; -public class DefaultTaskExecuteRunnable extends PriorityDelayTaskExecuteRunnable { +public class DefaultTaskExecutionRunnable extends PriorityDelayITaskExecutionRunnable { private final TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; - public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance, - TaskInstance taskInstance, - TaskExecutionContext taskExecutionContext, - TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager) { - super(workflowInstance, taskInstance, taskExecutionContext); + public DefaultTaskExecutionRunnable(TaskExecutionRunnableContext taskExecutionRunnableContext, + TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager) { + super(taskExecutionRunnableContext); this.taskExecuteRunnableOperatorManager = checkNotNull(taskExecuteRunnableOperatorManager); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultWorkflowExecutionRunnable.java similarity index 91% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultWorkflowExecutionRunnable.java index fa658c04f109..60d045972dcf 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultWorkflowExecutionRunnable.java @@ -39,7 +39,6 @@ import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; -import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.Command; @@ -66,11 +65,11 @@ import org.apache.dolphinscheduler.server.master.event.StateEvent; import org.apache.dolphinscheduler.server.master.event.StateEventHandleError; import org.apache.dolphinscheduler.server.master.event.StateEventHandleException; -import org.apache.dolphinscheduler.server.master.event.StateEventHandleFailure; -import org.apache.dolphinscheduler.server.master.event.StateEventHandler; -import org.apache.dolphinscheduler.server.master.event.StateEventHandlerManager; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent; +import org.apache.dolphinscheduler.server.master.events.EventDispatcher; +import org.apache.dolphinscheduler.server.master.events.WorkflowExecutionRunnableTriggerNextTaskEvent; +import org.apache.dolphinscheduler.server.master.events.WorkflowExecutionRunnableTriggeredEvent; import org.apache.dolphinscheduler.server.master.graph.IWorkflowGraph; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; @@ -81,7 +80,6 @@ import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; import org.apache.dolphinscheduler.service.command.CommandService; import org.apache.dolphinscheduler.service.cron.CronUtils; -import org.apache.dolphinscheduler.service.exceptions.CronParseException; import org.apache.dolphinscheduler.service.expand.CuringParamsService; import org.apache.dolphinscheduler.service.model.TaskNode; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -110,7 +108,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.stream.Collectors; -import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.springframework.beans.BeanUtils; @@ -119,11 +116,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -/** - * Workflow execute task, used to execute a workflow instance. - */ @Slf4j -public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable { +public class DefaultWorkflowExecutionRunnable implements IWorkflowExecutionRunnable { private final ProcessService processService; @@ -135,9 +129,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable { private final ProcessAlertManager processAlertManager; - private final IWorkflowExecuteContext workflowExecuteContext; - - private WorkflowRunnableStatus workflowRunnableStatus = WorkflowRunnableStatus.CREATED; + private final IWorkflowExecutionContext workflowExecuteContext; /** * submit failure nodes @@ -157,7 +149,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable { /** * TaskCode as Key, TaskExecuteRunnable as Value */ - private final Map taskExecuteRunnableMap = new ConcurrentHashMap<>(); + private final Map taskExecuteRunnableMap = new ConcurrentHashMap<>(); /** * valid task map, taskCode as key, taskId as value @@ -222,19 +214,20 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable { private final TaskGroupCoordinator taskGroupCoordinator; - public WorkflowExecuteRunnable( - @NonNull IWorkflowExecuteContext workflowExecuteContext, - @NonNull CommandService commandService, - @NonNull ProcessService processService, - @NonNull ProcessInstanceDao processInstanceDao, - @NonNull ProcessAlertManager processAlertManager, - @NonNull MasterConfig masterConfig, - @NonNull StateWheelExecuteThread stateWheelExecuteThread, - @NonNull CuringParamsService curingParamsService, - @NonNull TaskInstanceDao taskInstanceDao, - @NonNull DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory, - @NonNull ListenerEventAlertManager listenerEventAlertManager, - @NonNull TaskGroupCoordinator taskGroupCoordinator) { + private EventDispatcher eventDispatcher; + + public DefaultWorkflowExecutionRunnable(IWorkflowExecutionContext workflowExecuteContext, + CommandService commandService, + ProcessService processService, + ProcessInstanceDao processInstanceDao, + ProcessAlertManager processAlertManager, + MasterConfig masterConfig, + StateWheelExecuteThread stateWheelExecuteThread, + CuringParamsService curingParamsService, + TaskInstanceDao taskInstanceDao, + DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory, + ListenerEventAlertManager listenerEventAlertManager, + TaskGroupCoordinator taskGroupCoordinator) { this.processService = processService; this.commandService = commandService; this.processInstanceDao = processInstanceDao; @@ -250,76 +243,69 @@ public WorkflowExecuteRunnable( TaskMetrics.registerTaskPrepared(standByTaskInstancePriorityQueue::size); } - /** - * the process start nodes are submitted completely. - */ - public boolean isStart() { - return WorkflowRunnableStatus.STARTED == workflowRunnableStatus; - } - - /** - * handle event - */ - public void handleEvents() { - if (!isStart()) { - log.info( - "The workflow instance is not started, will not handle its state event, current state event size: {}", - stateEvents); - return; - } - int loopTimes = stateEvents.size() * 2; - for (int i = 0; i < loopTimes; i++) { - final StateEvent stateEvent = this.stateEvents.peek(); - if (stateEvent == null) { - return; - } - try { - LogUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), - stateEvent.getTaskInstanceId()); - // if state handle success then will remove this state, otherwise will retry this state next time. - // The state should always handle success except database error. - checkProcessInstance(stateEvent); - - StateEventHandler stateEventHandler = - StateEventHandlerManager.getStateEventHandler(stateEvent.getType()) - .orElseThrow(() -> new StateEventHandleError( - "Cannot find handler for the given state event")); - log.info("Begin to handle state event, {}", stateEvent); - if (stateEventHandler.handleStateEvent(this, stateEvent)) { - this.stateEvents.remove(stateEvent); - } - } catch (StateEventHandleError stateEventHandleError) { - log.error("State event handle error, will remove this event: {}", stateEvent, - stateEventHandleError); - this.stateEvents.remove(stateEvent); - ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); - } catch (StateEventHandleException stateEventHandleException) { - log.error("State event handle error, will retry this event: {}", - stateEvent, - stateEventHandleException); - ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); - } catch (StateEventHandleFailure stateEventHandleFailure) { - log.error("State event handle failed, will move event to the tail: {}", - stateEvent, - stateEventHandleFailure); - this.stateEvents.remove(stateEvent); - this.stateEvents.offer(stateEvent); - ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); - } catch (Exception e) { - // we catch the exception here, since if the state event handle failed, the state event will still - // keep - // in the stateEvents queue. - log.error("State event handle error, get a unknown exception, will retry this event: {}", - stateEvent, - e); - ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); - } finally { - LogUtils.removeWorkflowAndTaskInstanceIdMDC(); - } - } - } - - public IWorkflowExecuteContext getWorkflowExecuteContext() { + // /** + // * handle event + // */ + // public void handleEvents() { + // if (!isStart()) { + // log.info( + // "The workflow instance is not started, will not handle its state event, current state event size: {}", + // stateEvents); + // return; + // } + // int loopTimes = stateEvents.size() * 2; + // for (int i = 0; i < loopTimes; i++) { + // final StateEvent stateEvent = this.stateEvents.peek(); + // if (stateEvent == null) { + // return; + // } + // try { + // LogUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), + // stateEvent.getTaskInstanceId()); + // // if state handle success then will remove this state, otherwise will retry this state next time. + // // The state should always handle success except database error. + // checkProcessInstance(stateEvent); + // + // StateEventHandler stateEventHandler = + // StateEventHandlerManager.getStateEventHandler(stateEvent.getType()) + // .orElseThrow(() -> new StateEventHandleError( + // "Cannot find handler for the given state event")); + // log.info("Begin to handle state event, {}", stateEvent); + // if (stateEventHandler.handleStateEvent(this, stateEvent)) { + // this.stateEvents.remove(stateEvent); + // } + // } catch (StateEventHandleError stateEventHandleError) { + // log.error("State event handle error, will remove this event: {}", stateEvent, + // stateEventHandleError); + // this.stateEvents.remove(stateEvent); + // ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); + // } catch (StateEventHandleException stateEventHandleException) { + // log.error("State event handle error, will retry this event: {}", + // stateEvent, + // stateEventHandleException); + // ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); + // } catch (StateEventHandleFailure stateEventHandleFailure) { + // log.error("State event handle failed, will move event to the tail: {}", + // stateEvent, + // stateEventHandleFailure); + // this.stateEvents.remove(stateEvent); + // this.stateEvents.offer(stateEvent); + // ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); + // } catch (Exception e) { + // // we catch the exception here, since if the state event handle failed, the state event will still + // // keep + // // in the stateEvents queue. + // log.error("State event handle error, get a unknown exception, will retry this event: {}", + // stateEvent, + // e); + // ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); + // } finally { + // LogUtils.removeWorkflowAndTaskInstanceIdMDC(); + // } + // } + // } + + public IWorkflowExecutionContext getWorkflowExecuteContext() { return workflowExecuteContext; } @@ -332,16 +318,6 @@ public boolean addStateEvent(StateEvent stateEvent) { return true; } - public int eventSize() { - return this.stateEvents.size(); - } - - public void processStart() { - ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); - ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(workflowInstance.getId()); - this.listenerEventAlertManager.publishProcessStartListenerEvent(workflowInstance, projectUser); - } - public void taskStart(TaskInstance taskInstance) { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(workflowInstance.getId()); @@ -441,9 +417,8 @@ private void releaseTaskGroupIfNeeded(TaskInstance taskInstance) { /** * crate new task instance to retry, different objects from the original - * */ - private void retryTaskInstance(TaskInstance taskInstance) throws StateEventHandleException { + private void retryTaskInstance(TaskInstance taskInstance) { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); if (!taskInstance.taskCanRetry()) { return; @@ -659,6 +634,7 @@ private int createComplementDataCommand(Date scheduleDate) { command.setTestFlag(workflowInstance.getTestFlag()); int create = commandService.createCommand(command); processService.saveCommandTrigger(command.getId(), workflowInstance.getId()); + initTaskQueue(); return create; } @@ -667,38 +643,57 @@ private boolean needComplementProcess() { return workflowInstance.isComplementData() && Flag.NO == workflowInstance.getIsSubProcess(); } - /** - * ProcessInstance start entrypoint. - */ @Override - public WorkflowStartStatus startWorkflow() { + public void start() { + // build the dag + // init task queue + // ... + initTaskQueue(); + Integer workflowInstanceId = getWorkflowExecutionContext().getWorkflowInstance().getId(); + eventDispatcher + .dispatchEvent(new WorkflowExecutionRunnableTriggeredEvent(workflowInstanceId)); + eventDispatcher + .dispatchEvent(new WorkflowExecutionRunnableTriggerNextTaskEvent(workflowInstanceId, null)); + } + + @Override + public void triggerNextTasks(Long parentTaskCode) { + submitPostNode(null); + } + + @Override + public void kill() { - try { - ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); - LogUtils.setWorkflowInstanceIdMDC(workflowInstance.getId()); - if (isStart()) { - // This case should not been happened - log.warn("The workflow has already been started, current state: {}", workflowRunnableStatus); - return WorkflowStartStatus.DUPLICATED_SUBMITTED; - } - if (workflowRunnableStatus == WorkflowRunnableStatus.CREATED) { - initTaskQueue(); - workflowRunnableStatus = WorkflowRunnableStatus.INITIALIZE_QUEUE; - log.info("workflowStatue changed to :{}", workflowRunnableStatus); - } - if (workflowRunnableStatus == WorkflowRunnableStatus.INITIALIZE_QUEUE) { - processStart(); - submitPostNode(null); - workflowRunnableStatus = WorkflowRunnableStatus.STARTED; - log.info("workflowStatue changed to :{}", workflowRunnableStatus); - } - return WorkflowStartStatus.SUCCESS; - } catch (Exception e) { - log.error("Start workflow error", e); - return WorkflowStartStatus.FAILED; - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); - } + } + + @Override + public void pause() { + + } + + @Override + public void timeout() { + + } + + @Override + public ITaskExecutionRunnable getTaskExecutionRunnableById(int taskInstanceId) { + return null; + } + + @Override + public ITaskExecutionRunnable getTaskExecutionRunnableByCode(long taskCode) { + return null; + } + + @Override + public Collection getActiveTaskExecutionRunnable() { + return null; + } + + @Override + public WorkflowExecutionContext getWorkflowExecutionContext() { + return null; } /** @@ -773,7 +768,7 @@ public void checkSerialProcess(ProcessDefinition processDefinition) { commandService.createCommand(command); } - private void initTaskQueue() throws StateEventHandleException, CronParseException { + private void initTaskQueue() { taskFailedSubmit = false; // do we need to clear? @@ -914,7 +909,7 @@ private boolean executeTask(TaskInstance taskInstance) { // in a dag, only one taskInstance is valid per taskCode, so need to set the old taskInstance invalid try { LogUtils.setTaskInstanceIdMDC(taskInstance.getId()); - DefaultTaskExecuteRunnable taskExecuteRunnable = + DefaultTaskExecutionRunnable taskExecuteRunnable = defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance); if (validTaskMap.containsKey(taskInstance.getTaskCode())) { int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode()); @@ -962,9 +957,9 @@ private boolean executeTask(TaskInstance taskInstance) { * there is no need to dispatch it */ @VisibleForTesting - void tryToDispatchTaskInstance(TaskInstance taskInstance, TaskExecuteRunnable taskExecuteRunnable) { + void tryToDispatchTaskInstance(TaskInstance taskInstance, ITaskExecutionRunnable ITaskExecutionRunnable) { if (!taskInstance.getState().isFinished()) { - taskExecuteRunnable.dispatch(); + ITaskExecutionRunnable.dispatch(); } else { if (workflowExecuteContext.getWorkflowInstance().isBlocked()) { TaskStateEvent processBlockEvent = TaskStateEvent.builder() @@ -1267,7 +1262,7 @@ private List getValidTaskList() { return validTaskInstanceList; } - private void submitPostNode(Long parentNodeCode) throws StateEventHandleException { + private void submitPostNode(Long parentNodeCode) { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); DAG dag = workflowExecuteContext.getWorkflowGraph().getDag(); @@ -1629,8 +1624,9 @@ private WorkflowExecutionStatus processReadyPause() { */ private WorkflowExecutionStatus processReadyBlock() { if (taskExecuteRunnableMap.size() > 0) { - for (DefaultTaskExecuteRunnable taskExecuteRunnable : taskExecuteRunnableMap.values()) { - if (!TASK_TYPE_BLOCKING.equals(taskExecuteRunnable.getTaskInstance().getTaskType())) { + for (DefaultTaskExecutionRunnable taskExecuteRunnable : taskExecuteRunnableMap.values()) { + if (!TASK_TYPE_BLOCKING.equals( + taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getTaskType())) { taskExecuteRunnable.pause(); } } @@ -1732,7 +1728,7 @@ private boolean isComplementEnd() { * updateProcessInstance process instance state * after each batch of tasks is executed, the status of the process instance is updated */ - private void updateProcessInstanceState() throws StateEventHandleException { + private void updateProcessInstanceState() { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); WorkflowExecutionStatus state = getProcessInstanceState(workflowInstance); if (workflowInstance.getState() != state) { @@ -1763,7 +1759,7 @@ public void updateProcessInstanceState(WorkflowStateEvent stateEvent) throws Sta updateWorkflowInstanceStatesToDB(state); } - private void updateWorkflowInstanceStatesToDB(WorkflowExecutionStatus newStates) throws StateEventHandleException { + private void updateWorkflowInstanceStatesToDB(WorkflowExecutionStatus newStates) { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); WorkflowExecutionStatus originStates = workflowInstance.getState(); if (originStates != newStates) { @@ -1775,14 +1771,7 @@ private void updateWorkflowInstanceStatesToDB(WorkflowExecutionStatus newStates) if (newStates.isFinished()) { workflowInstance.setEndTime(new Date()); } - try { - processInstanceDao.performTransactionalUpsert(workflowInstance); - } catch (Exception ex) { - // recover the status - workflowInstance.setStateWithDesc(originStates, "recover state by DB error"); - workflowInstance.setEndTime(null); - throw new StateEventHandleException("Update process instance status to DB error", ex); - } + processInstanceDao.performTransactionalUpsert(workflowInstance); } } @@ -1862,13 +1851,15 @@ public void killAllTasks() { if (taskInstance == null || taskInstance.getState().isFinished()) { continue; } - DefaultTaskExecuteRunnable defaultTaskExecuteRunnable = taskExecuteRunnableMap.get(taskCode); + DefaultTaskExecutionRunnable defaultTaskExecuteRunnable = taskExecuteRunnableMap.get(taskCode); defaultTaskExecuteRunnable.kill(); - if (defaultTaskExecuteRunnable.getTaskInstance().getState().isFinished()) { + if (defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getState() + .isFinished()) { TaskStateEvent taskStateEvent = TaskStateEvent.builder() .processInstanceId(workflowInstance.getId()) .taskInstanceId(taskInstance.getId()) - .status(defaultTaskExecuteRunnable.getTaskInstance().getState()) + .status(defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance() + .getState()) .type(StateEventType.TASK_STATE_CHANGE) .build(); this.addStateEvent(taskStateEvent); @@ -1886,7 +1877,7 @@ public boolean workFlowFinish() { /** * handling the list of tasks to be submitted */ - public void submitStandByTask() throws StateEventHandleException { + public void submitStandByTask() { ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance(); TaskInstance task; while ((task = standByTaskInstancePriorityQueue.peek()) != null) { @@ -2035,11 +2026,11 @@ public Set getCompleteTaskCodes() { return completeTaskSet; } - public Map getTaskExecuteRunnableMap() { + public Map getTaskExecuteRunnableMap() { return taskExecuteRunnableMap; } - public Optional getTaskExecuteRunnableById(Integer taskInstanceId) { + public Optional getTaskExecuteRunnableById(Integer taskInstanceId) { if (taskInstanceId == null) { throw new IllegalArgumentException("taskInstanceId can't be null"); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java index cad36e8b6802..ae5ec9cb2621 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager; import java.util.concurrent.TimeUnit; @@ -36,7 +36,7 @@ public class EventExecuteService extends BaseDaemonThread { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private StreamTaskInstanceExecCacheManager streamTaskInstanceExecCacheManager; @@ -76,11 +76,12 @@ public void run() { } private void workflowEventHandler() { - for (WorkflowExecuteRunnable workflowExecuteThread : this.processInstanceExecCacheManager.getAll()) { + for (IWorkflowExecutionRunnable workflowExecutionRunnable : this.IWorkflowExecuteRunnableRepository.getAll()) { try { LogUtils.setWorkflowInstanceIdMDC( - workflowExecuteThread.getWorkflowExecuteContext().getWorkflowInstance().getId()); - workflowExecuteThreadPool.executeEvent(workflowExecuteThread); + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance().getId()); + // todo: remove this class, the event should be handled by the event bus + // workflowExecuteThreadPool.executeEvent(workflowExecutionRunnable); } finally { LogUtils.removeWorkflowInstanceIdMDC(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java index 7e0d683571de..f12b3374b1f6 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java @@ -24,20 +24,20 @@ import org.springframework.stereotype.Component; /** - * The class is used to store {@link TaskExecuteRunnable} which needs to be dispatched. The {@link TaskExecuteRunnable} will be stored in a {@link DelayQueue}, - * if the {@link TaskExecuteRunnable}'s delay time is 0, then it will be consumed by {@link GlobalTaskDispatchWaitingQueueLooper}. + * The class is used to store {@link ITaskExecutionRunnable} which needs to be dispatched. The {@link ITaskExecutionRunnable} will be stored in a {@link DelayQueue}, + * if the {@link ITaskExecutionRunnable}'s delay time is 0, then it will be consumed by {@link GlobalTaskDispatchWaitingQueueLooper}. */ @Slf4j @Component public class GlobalTaskDispatchWaitingQueue { - private final DelayQueue queue = new DelayQueue<>(); + private final DelayQueue queue = new DelayQueue<>(); - public void submitTaskExecuteRunnable(DefaultTaskExecuteRunnable priorityTaskExecuteRunnable) { + public void submitTaskExecuteRunnable(DefaultTaskExecutionRunnable priorityTaskExecuteRunnable) { queue.put(priorityTaskExecuteRunnable); } - public DefaultTaskExecuteRunnable takeTaskExecuteRunnable() throws InterruptedException { + public DefaultTaskExecutionRunnable takeTaskExecuteRunnable() throws InterruptedException { return queue.take(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java index a1f4b28783a3..84821d722dbd 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java @@ -63,7 +63,7 @@ public synchronized void start() { @Override public void run() { - DefaultTaskExecuteRunnable defaultTaskExecuteRunnable; + DefaultTaskExecutionRunnable defaultTaskExecuteRunnable; while (RUNNING_FLAG.get()) { try { defaultTaskExecuteRunnable = globalTaskDispatchWaitingQueue.takeTaskExecuteRunnable(); @@ -74,16 +74,19 @@ public void run() { } try { TaskDispatcher taskDispatcher = - taskDispatchFactory.getTaskDispatcher(defaultTaskExecuteRunnable.getTaskInstance()); + taskDispatchFactory.getTaskDispatcher( + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance()); taskDispatcher.dispatchTask(defaultTaskExecuteRunnable); DISPATCHED_CONSECUTIVE_FAILURE_TIMES.set(0); } catch (Exception e) { - defaultTaskExecuteRunnable.getTaskExecutionContext().increaseDispatchFailTimes(); + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext() + .increaseDispatchFailTimes(); globalTaskDispatchWaitingQueue.submitTaskExecuteRunnable(defaultTaskExecuteRunnable); if (DISPATCHED_CONSECUTIVE_FAILURE_TIMES.incrementAndGet() > MAX_DISPATCHED_FAILED_TIMES) { ThreadUtils.sleep(10 * 1000L); } - log.error("Dispatch Task: {} failed", defaultTaskExecuteRunnable.getTaskInstance().getName(), e); + log.error("Dispatch Task: {} failed", + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getName(), e); } } log.info("GlobalTaskDispatchWaitingQueueLooper started..."); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IEventfulExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IEventfulExecutionRunnable.java new file mode 100644 index 000000000000..716130274b98 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IEventfulExecutionRunnable.java @@ -0,0 +1,10 @@ +package org.apache.dolphinscheduler.server.master.runner; + +import org.apache.dolphinscheduler.server.master.events.IEventRepository; + +public interface IEventfulExecutionRunnable { + + default IEventRepository getEventRepository() { + return null; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/ITaskExecutionRunnable.java similarity index 65% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnable.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/ITaskExecutionRunnable.java index 8f661896171e..033a5e21aeb8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/ITaskExecutionRunnable.java @@ -17,27 +17,31 @@ package org.apache.dolphinscheduler.server.master.runner; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; - /** - * This interface is used to define a task which is executing. - * todo: split to MasterTaskExecuteRunnable and WorkerTaskExecuteRunnable + * The ITaskExecutionRunnable represent a running task instance, it is responsible for operate the task instance. e.g. dispatch, kill, pause, timeout. */ -public interface TaskExecuteRunnable { +public interface ITaskExecutionRunnable { + /** + * Dispatch the task instance. + */ void dispatch(); + /** + * Kill the task instance. + */ void kill(); + /** + * Pause the task instance. + */ void pause(); + /** + * Timeout the task instance. + */ void timeout(); - ProcessInstance getWorkflowInstance(); - - TaskInstance getTaskInstance(); + TaskExecutionRunnableContext getTaskExecutionRunnableContext(); - TaskExecutionContext getTaskExecutionContext(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionContext.java similarity index 96% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteContext.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionContext.java index 9c012a235517..621c93b58108 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecuteContext.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionContext.java @@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.server.master.graph.IWorkflowGraph; // todo: Add method to manage the task instance -public interface IWorkflowExecuteContext { +public interface IWorkflowExecutionContext { ProcessDefinition getWorkflowDefinition(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionRunnable.java new file mode 100644 index 000000000000..fcad2f7f8e92 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/IWorkflowExecutionRunnable.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner; + +import org.apache.dolphinscheduler.server.master.events.IEvent; + +import java.util.Collection; + +/** + * The IWorkflowExecuteRunnable represent a running workflow instance, it is responsible for operate the workflow instance. e.g. start, kill, pause, timeout. + */ +public interface IWorkflowExecutionRunnable extends IEventfulExecutionRunnable { + + /** + * Start the workflow instance. + */ + void start(); + + /** + * Trigger the next tasks which are depend on the given task. + * + * @param parentTaskCode the parent task code + */ + void triggerNextTasks(Long parentTaskCode); + + /** + * Kill the workflow instance. + */ + void kill(); + + /** + * Pause the workflow instance. + */ + void pause(); + + /** + * Timeout the workflow instance. + */ + void timeout(); + + // todo: move to TaskExecutionRunnableRepository + ITaskExecutionRunnable getTaskExecutionRunnableById(int taskInstanceId); + + ITaskExecutionRunnable getTaskExecutionRunnableByCode(long taskCode); + + Collection getActiveTaskExecutionRunnable(); + + /** + * Get the workflow execution context. + * + * @return the workflow execution context + */ + WorkflowExecutionContext getWorkflowExecutionContext(); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java index 2fddd9438474..cd7c5c1b437c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java @@ -25,12 +25,12 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.meter.metrics.MetricsProvider; import org.apache.dolphinscheduler.meter.metrics.SystemMetrics; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterServerLoadProtection; -import org.apache.dolphinscheduler.server.master.event.WorkflowEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowEventQueue; -import org.apache.dolphinscheduler.server.master.event.WorkflowEventType; +import org.apache.dolphinscheduler.server.master.events.EventDispatcher; +import org.apache.dolphinscheduler.server.master.events.WorkflowExecutionRunnableTriggerEvent; import org.apache.dolphinscheduler.server.master.exception.MasterException; import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException; import org.apache.dolphinscheduler.server.master.metrics.MasterServerMetrics; @@ -63,7 +63,7 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl private MasterConfig masterConfig; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; @Autowired private WorkflowExecuteRunnableFactory workflowExecuteRunnableFactory; @@ -83,6 +83,9 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl @Autowired private MetricsProvider metricsProvider; + @Autowired + private EventDispatcher eventDispatcher; + protected MasterSchedulerBootstrap() { super("MasterCommandLoopThread"); } @@ -132,31 +135,32 @@ public void run() { continue; } - commands.parallelStream() - .forEach(command -> { - try { - Optional workflowExecuteRunnableOptional = - workflowExecuteRunnableFactory.createWorkflowExecuteRunnable(command); - if (!workflowExecuteRunnableOptional.isPresent()) { - log.warn( - "The command execute success, will not trigger a WorkflowExecuteRunnable, this workflowInstance might be in serial mode"); - return; - } - WorkflowExecuteRunnable workflowExecuteRunnable = workflowExecuteRunnableOptional.get(); - ProcessInstance processInstance = workflowExecuteRunnable - .getWorkflowExecuteContext().getWorkflowInstance(); - if (processInstanceExecCacheManager.contains(processInstance.getId())) { - log.error( - "The workflow instance is already been cached, this case shouldn't be happened"); - } - processInstanceExecCacheManager.cache(processInstance.getId(), workflowExecuteRunnable); - workflowEventQueue.addEvent( - new WorkflowEvent(WorkflowEventType.START_WORKFLOW, processInstance.getId())); - } catch (WorkflowCreateException workflowCreateException) { - log.error("Master handle command {} error ", command.getId(), workflowCreateException); - commandService.moveToErrorCommand(command, workflowCreateException.toString()); - } - }); + for (Command command : commands) { + try { + Optional workflowExecuteRunnableOptional = + workflowExecuteRunnableFactory.createWorkflowExecuteRunnable(command); + if (!workflowExecuteRunnableOptional.isPresent()) { + log.warn( + "Transform command: {} to WorkflowExecutionRunnable failed, the workflowInstance might be in serial mode", + command); + return; + } + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableOptional.get(); + ProcessInstance processInstance = + workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(); + if (workflowExecuteRunnableRepository.contains(processInstance.getId())) { + log.error("The workflow instance is already been cached, this case shouldn't be happened"); + return; + } + workflowExecuteRunnableRepository.cache(processInstance.getId(), workflowExecuteRunnable); + eventDispatcher + .dispatchEvent(new WorkflowExecutionRunnableTriggerEvent(processInstance.getId())); + } catch (WorkflowCreateException workflowCreateException) { + log.error("Master handle command {} error ", command.getId(), workflowCreateException); + commandService.moveToErrorCommand(command, workflowCreateException.toString()); + } + } MasterServerMetrics.incMasterConsumeCommand(commands.size()); } catch (InterruptedException interruptedException) { log.warn("Master schedule bootstrap interrupted, close the loop", interruptedException); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayITaskExecutionRunnable.java similarity index 53% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayTaskExecuteRunnable.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayITaskExecutionRunnable.java index 255ec6c8ac2c..c6835610d35c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayTaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/PriorityDelayITaskExecutionRunnable.java @@ -18,26 +18,21 @@ package org.apache.dolphinscheduler.server.master.runner; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import java.util.concurrent.Delayed; import java.util.concurrent.TimeUnit; -public abstract class PriorityDelayTaskExecuteRunnable extends BaseTaskExecuteRunnable implements Delayed { +public abstract class PriorityDelayITaskExecutionRunnable extends BaseITaskExecutionRunnable implements Delayed { - public PriorityDelayTaskExecuteRunnable(ProcessInstance workflowInstance, - TaskInstance taskInstance, - TaskExecutionContext taskExecutionContext) { - super(workflowInstance, taskInstance, taskExecutionContext); + public PriorityDelayITaskExecutionRunnable(TaskExecutionRunnableContext taskExecutionRunnableContext) { + super(taskExecutionRunnableContext); } @Override public long getDelay(TimeUnit unit) { return unit.convert( - DateUtils.getRemainTime(taskExecutionContext.getFirstSubmitTime(), - taskExecutionContext.getDelayTime() * 60L), + DateUtils.getRemainTime(taskExecutionRunnableContext.getTaskExecutionContext().getFirstSubmitTime(), + taskExecutionRunnableContext.getTaskExecutionContext().getDelayTime() * 60L), TimeUnit.SECONDS); } @@ -51,35 +46,41 @@ public int compareTo(Delayed o) { if (delayTimeCompareResult != 0) { return delayTimeCompareResult; } - PriorityDelayTaskExecuteRunnable other = (PriorityDelayTaskExecuteRunnable) o; + PriorityDelayITaskExecutionRunnable other = (PriorityDelayITaskExecutionRunnable) o; // the smaller dispatch fail times, the higher priority - int dispatchFailTimesCompareResult = taskExecutionContext.getDispatchFailTimes() - - other.getTaskExecutionContext().getDispatchFailTimes(); + int dispatchFailTimesCompareResult = + taskExecutionRunnableContext.getTaskExecutionContext().getDispatchFailTimes() + - other.getTaskExecutionRunnableContext().getTaskExecutionContext().getDispatchFailTimes(); if (dispatchFailTimesCompareResult != 0) { return dispatchFailTimesCompareResult; } - int workflowInstancePriorityCompareResult = workflowInstance.getProcessInstancePriority().getCode() - - other.getWorkflowInstance().getProcessInstancePriority().getCode(); + int workflowInstancePriorityCompareResult = taskExecutionRunnableContext.getWorkflowInstance() + .getProcessInstancePriority().getCode() + - other.getTaskExecutionRunnableContext().getWorkflowInstance().getProcessInstancePriority().getCode(); if (workflowInstancePriorityCompareResult != 0) { return workflowInstancePriorityCompareResult; } - long workflowInstanceIdCompareResult = workflowInstance.getId().compareTo(other.getWorkflowInstance().getId()); + long workflowInstanceIdCompareResult = taskExecutionRunnableContext.getWorkflowInstance().getId() + .compareTo(other.getTaskExecutionRunnableContext().getWorkflowInstance().getId()); if (workflowInstanceIdCompareResult != 0) { return workflowInstancePriorityCompareResult; } - int taskInstancePriorityCompareResult = taskInstance.getTaskInstancePriority().getCode() - - other.getTaskInstance().getTaskInstancePriority().getCode(); + int taskInstancePriorityCompareResult = + taskExecutionRunnableContext.getTaskInstance().getTaskInstancePriority().getCode() + - other.getTaskExecutionRunnableContext().getTaskInstance().getTaskInstancePriority().getCode(); if (taskInstancePriorityCompareResult != 0) { return taskInstancePriorityCompareResult; } // larger number, higher priority int taskGroupPriorityCompareResult = - taskInstance.getTaskGroupPriority() - other.getTaskInstance().getTaskGroupPriority(); + taskExecutionRunnableContext.getTaskInstance().getTaskGroupPriority() + - other.getTaskExecutionRunnableContext().getTaskInstance().getTaskGroupPriority(); if (taskGroupPriorityCompareResult != 0) { return -taskGroupPriorityCompareResult; } // The task instance shouldn't be equals - return taskInstance.getId().compareTo(other.getTaskInstance().getId()); + return taskExecutionRunnableContext.getTaskInstance().getId() + .compareTo(other.getTaskExecutionRunnableContext().getTaskInstance().getId()); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java index 667eb4ecbb28..7a628f2dd7b1 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java @@ -28,13 +28,12 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent; import org.apache.dolphinscheduler.server.master.runner.task.TaskInstanceKey; -import java.util.Optional; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; @@ -81,7 +80,7 @@ public class StateWheelExecuteThread extends BaseDaemonThread { private WorkflowExecuteThreadPool workflowExecuteThreadPool; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; protected StateWheelExecuteThread() { super("StateWheelExecuteThread"); @@ -132,8 +131,9 @@ private void checkProcess4Timeout() { for (Integer processInstanceId : processInstanceTimeoutCheckList) { try { LogUtils.setWorkflowInstanceIdMDC(processInstanceId); - WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId( - processInstanceId); + IWorkflowExecutionRunnable workflowExecuteThread = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId( + processInstanceId); if (workflowExecuteThread == null) { log.warn( "Check workflow timeout failed, can not find workflowExecuteThread from cache manager, will remove this workflowInstance from check list"); @@ -141,7 +141,7 @@ private void checkProcess4Timeout() { continue; } ProcessInstance processInstance = - workflowExecuteThread.getWorkflowExecuteContext().getWorkflowInstance(); + workflowExecuteThread.getWorkflowExecutionContext().getWorkflowInstance(); if (processInstance == null) { log.warn("Check workflow timeout failed, the workflowInstance is null"); continue; @@ -225,17 +225,17 @@ private void checkTask4Timeout() { int processInstanceId = taskInstanceKey.getProcessInstanceId(); long taskCode = taskInstanceKey.getTaskCode(); - WorkflowExecuteRunnable workflowExecuteThread = - processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); - if (workflowExecuteThread == null) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); + if (workflowExecutionRunnable == null) { log.warn( "Check task instance timeout failed, can not find workflowExecuteThread from cache manager, will remove this check task"); taskInstanceTimeoutCheckList.remove(taskInstanceKey); continue; } - Optional taskInstanceOptional = - workflowExecuteThread.getActiveTaskInstanceByTaskCode(taskCode); - if (!taskInstanceOptional.isPresent()) { + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionRunnable.getTaskExecutionRunnableByCode(taskCode); + if (taskExecutionRunnable == null) { log.warn( "Check task instance timeout failed, can not get taskInstance from workflowExecuteThread, taskCode: {}" + "will remove this check task", @@ -243,7 +243,7 @@ private void checkTask4Timeout() { taskInstanceTimeoutCheckList.remove(taskInstanceKey); continue; } - TaskInstance taskInstance = taskInstanceOptional.get(); + TaskInstance taskInstance = taskExecutionRunnable.getTaskExecutionRunnableContext().getTaskInstance(); if (TimeoutFlag.OPEN == taskInstance.getTaskDefine().getTimeoutFlag()) { long timeRemain = DateUtils.getRemainTime(taskInstance.getStartTime(), TimeUnit.MINUTES.toSeconds(taskInstance.getTaskDefine().getTimeout())); @@ -272,10 +272,10 @@ private void checkTask4Retry() { try { LogUtils.setWorkflowInstanceIdMDC(processInstanceId); - WorkflowExecuteRunnable workflowExecuteThread = - processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + IWorkflowExecutionRunnable workflowExecutionRunnable = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); - if (workflowExecuteThread == null) { + if (workflowExecutionRunnable == null) { log.warn( "Task instance retry check failed, can not find workflowExecuteThread from cache manager, " + "will remove this check task"); @@ -283,10 +283,10 @@ private void checkTask4Retry() { continue; } - Optional taskInstanceOptional = - workflowExecuteThread.getRetryTaskInstanceByTaskCode(taskCode); + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionRunnable.getTaskExecutionRunnableByCode(taskCode); ProcessInstance processInstance = - workflowExecuteThread.getWorkflowExecuteContext().getWorkflowInstance(); + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); if (processInstance.getState().isReadyStop()) { log.warn( @@ -296,14 +296,14 @@ private void checkTask4Retry() { break; } - if (!taskInstanceOptional.isPresent()) { + if (taskExecutionRunnable == null) { log.warn( "Task instance retry check failed, can not find taskInstance from workflowExecuteThread, will remove this check"); taskInstanceRetryCheckList.remove(taskInstanceKey); continue; } - TaskInstance taskInstance = taskInstanceOptional.get(); + TaskInstance taskInstance = taskExecutionRunnable.getTaskExecutionRunnableContext().getTaskInstance(); // We check the status to avoid when we do worker failover we submit a failover task, this task may be // resubmit by this // thread diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java index 2f61507e727b..e3966a752fed 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java @@ -146,7 +146,7 @@ public void run() { processDefinition = processService.findProcessDefinition(processDefinitionCode, processDefinitionVersion); try { - DefaultTaskExecuteRunnable taskExecuteRunnable = + DefaultTaskExecutionRunnable taskExecuteRunnable = defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance); workerTaskDispatcher.dispatchTask(taskExecuteRunnable); } catch (Exception e) { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java index 43ee971160a8..4a4248f7e907 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java @@ -21,12 +21,19 @@ import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; /** - * Use to create TaskExecuteRunnable + * The factory use to create {@link ITaskExecutionRunnable}. * - * @param TaskExecuteRunnable + * @param TaskExecuteRunnable type */ -public interface TaskExecuteRunnableFactory { +public interface TaskExecuteRunnableFactory { + /** + * Create {@link ITaskExecutionRunnable}. + * + * @param taskInstance task instance + * @return TaskExecuteRunnable + * @throws TaskExecuteRunnableCreateException If create TaskExecuteRunnable failed + */ T createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java index ab1806ff6720..742c8187d127 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java @@ -60,7 +60,6 @@ import org.apache.dolphinscheduler.plugin.task.spark.SparkParameters; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException; import org.apache.dolphinscheduler.service.expand.CuringParamsService; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; @@ -102,7 +101,7 @@ public class TaskExecutionContextFactory { @Autowired private HikariDataSource hikariDataSource; - public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) throws TaskExecutionContextCreateException { + public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) { ProcessInstance workflowInstance = taskInstance.getProcessInstance(); ResourceParametersHelper resources = diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContext.java new file mode 100644 index 000000000000..b9afc4dedec8 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContext.java @@ -0,0 +1,21 @@ +package org.apache.dolphinscheduler.server.master.runner; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class TaskExecutionRunnableContext { + + private ProcessInstance workflowInstance; + private TaskInstance taskInstance; + private TaskExecutionContext taskExecutionContext; +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContextFactory.java new file mode 100644 index 000000000000..cdf7d84f2f33 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionRunnableContextFactory.java @@ -0,0 +1,36 @@ +package org.apache.dolphinscheduler.server.master.runner; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +/** + * The factory used to create {@link TaskExecutionRunnableContext} + */ +@Component +public class TaskExecutionRunnableContextFactory { + + @Autowired + private IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private TaskExecutionContextFactory taskExecutionContextFactory; + + /** + * Create TaskExecutionRunnableContext + * + * @param taskInstance task instance + * @return TaskExecutionRunnableContext + */ + public TaskExecutionRunnableContext createTaskExecutionRunnableContext(TaskInstance taskInstance) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(taskInstance.getProcessInstanceId()); + return TaskExecutionRunnableContext.builder() + .workflowInstance(workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstance()) + .taskInstance(taskInstance) + .taskExecutionContext(taskExecutionContextFactory.createTaskExecutionContext(taskInstance)) + .build(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java index 89810aea1f36..88a5671a0d17 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContextFactory.java @@ -52,7 +52,7 @@ public class WorkflowExecuteContextFactory { @Autowired private MasterConfig masterConfig; - public Optional createWorkflowExecuteRunnableContext(Command command) throws Exception { + public Optional createWorkflowExecuteRunnableContext(Command command) throws Exception { Optional workflowInstanceOptional = createWorkflowInstance(command); if (!workflowInstanceOptional.isPresent()) { return Optional.empty(); @@ -64,7 +64,7 @@ public Optional createWorkflowExecuteRunnableContext(Co IWorkflowGraph workflowGraph = workflowGraphFactory.createWorkflowGraph(workflowInstance); - return Optional.of(new WorkflowExecuteContext(workflowDefinition, workflowInstance, workflowGraph)); + return Optional.of(new WorkflowExecutionContext(workflowDefinition, workflowInstance, workflowGraph)); } private Optional createWorkflowInstance(Command command) throws CronParseException { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java index 7caed6ba757d..d7eece122463 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java @@ -77,23 +77,24 @@ public class WorkflowExecuteRunnableFactory { @Autowired private TaskGroupCoordinator taskGroupCoordinator; - public Optional createWorkflowExecuteRunnable(Command command) throws WorkflowCreateException { + public Optional createWorkflowExecuteRunnable(Command command) throws WorkflowCreateException { try { - Optional workflowExecuteRunnableContextOptional = + Optional workflowExecuteRunnableContextOptional = workflowExecuteContextFactory.createWorkflowExecuteRunnableContext(command); - return workflowExecuteRunnableContextOptional.map(iWorkflowExecuteContext -> new WorkflowExecuteRunnable( - iWorkflowExecuteContext, - commandService, - processService, - processInstanceDao, - processAlertManager, - masterConfig, - stateWheelExecuteThread, - curingGlobalParamsService, - taskInstanceDao, - defaultTaskExecuteRunnableFactory, - listenerEventAlertManager, - taskGroupCoordinator)); + return workflowExecuteRunnableContextOptional + .map(iWorkflowExecuteContext -> new DefaultWorkflowExecutionRunnable( + iWorkflowExecuteContext, + commandService, + processService, + processInstanceDao, + processAlertManager, + masterConfig, + stateWheelExecuteThread, + curingGlobalParamsService, + taskInstanceDao, + defaultTaskExecuteRunnableFactory, + listenerEventAlertManager, + taskGroupCoordinator)); } catch (Exception ex) { throw new WorkflowCreateException("Create WorkflowExecuteRunnable failed", ex); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java index 1aa423286566..6a9f5f6d5854 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java @@ -17,8 +17,7 @@ package org.apache.dolphinscheduler.server.master.runner; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.StateEvent; @@ -31,11 +30,9 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor; import org.springframework.stereotype.Component; -import org.springframework.util.concurrent.ListenableFuture; -import org.springframework.util.concurrent.ListenableFutureCallback; /** - * Used to execute {@link WorkflowExecuteRunnable}. + * Used to execute {@link DefaultWorkflowExecutionRunnable}. */ @Component @Slf4j @@ -45,7 +42,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor { private MasterConfig masterConfig; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Autowired private StateWheelExecuteThread stateWheelExecuteThread; @@ -53,7 +50,8 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor { /** * multi-thread filter, avoid handling workflow at the same time */ - private ConcurrentHashMap multiThreadFilterMap = new ConcurrentHashMap<>(); + private ConcurrentHashMap multiThreadFilterMap = + new ConcurrentHashMap<>(); @PostConstruct private void init() { @@ -67,68 +65,15 @@ private void init() { * submit state event */ public void submitStateEvent(StateEvent stateEvent) { - WorkflowExecuteRunnable workflowExecuteThread = - processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId()); + IWorkflowExecutionRunnable workflowExecuteThread = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(stateEvent.getProcessInstanceId()); if (workflowExecuteThread == null) { log.warn("Submit state event error, cannot from workflowExecuteThread from cache manager, stateEvent:{}", stateEvent); return; } - workflowExecuteThread.addStateEvent(stateEvent); + // workflowExecuteThread.addStateEvent(stateEvent); log.info("Submit state event success, stateEvent: {}", stateEvent); } - /** - * Handle the events belong to the given workflow. - */ - public void executeEvent(final WorkflowExecuteRunnable workflowExecuteThread) { - if (!workflowExecuteThread.isStart() || workflowExecuteThread.eventSize() == 0) { - return; - } - IWorkflowExecuteContext workflowExecuteRunnableContext = - workflowExecuteThread.getWorkflowExecuteContext(); - Integer workflowInstanceId = workflowExecuteRunnableContext.getWorkflowInstance().getId(); - - if (multiThreadFilterMap.containsKey(workflowInstanceId)) { - log.debug("The workflow has been executed by another thread"); - return; - } - multiThreadFilterMap.put(workflowInstanceId, workflowExecuteThread); - ListenableFuture future = this.submitListenable(workflowExecuteThread::handleEvents); - future.addCallback(new ListenableFutureCallback() { - - @Override - public void onFailure(Throwable ex) { - LogUtils.setWorkflowInstanceIdMDC(workflowInstanceId); - try { - log.error("Workflow instance events handle failed", ex); - multiThreadFilterMap.remove(workflowInstanceId); - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); - } - } - - @Override - public void onSuccess(Object result) { - try { - LogUtils.setWorkflowInstanceIdMDC( - workflowExecuteThread.getWorkflowExecuteContext().getWorkflowInstance().getId()); - if (workflowExecuteThread.workFlowFinish() && workflowExecuteThread.eventSize() == 0) { - stateWheelExecuteThread - .removeProcess4TimeoutCheck(workflowExecuteThread.getWorkflowExecuteContext() - .getWorkflowInstance().getId()); - processInstanceExecCacheManager.removeByProcessInstanceId(workflowInstanceId); - log.info("Workflow instance is finished."); - } - } catch (Exception e) { - log.error("Workflow instance is finished, but notify changed error", e); - } finally { - // make sure the process has been removed from multiThreadFilterMap - multiThreadFilterMap.remove(workflowInstanceId); - LogUtils.removeWorkflowInstanceIdMDC(); - } - } - }); - } - } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecutionContext.java similarity index 86% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContext.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecutionContext.java index fd6c4d44db6e..46151ecb892f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContext.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecutionContext.java @@ -25,7 +25,7 @@ import lombok.Getter; -public class WorkflowExecuteContext implements IWorkflowExecuteContext { +public class WorkflowExecutionContext implements IWorkflowExecutionContext { @Getter private final ProcessDefinition workflowDefinition; @@ -39,9 +39,9 @@ public class WorkflowExecuteContext implements IWorkflowExecuteContext { @Getter private final IWorkflowGraph workflowGraph; - public WorkflowExecuteContext(ProcessDefinition workflowDefinition, - ProcessInstance workflowInstance, - IWorkflowGraph workflowGraph) { + public WorkflowExecutionContext(ProcessDefinition workflowDefinition, + ProcessInstance workflowInstance, + IWorkflowGraph workflowGraph) { checkNotNull(workflowDefinition, "workflowDefinition is null"); checkNotNull(workflowInstance, "workflowInstance is null"); checkNotNull(workflowGraph, "workflowGraph is null"); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java index cca711297553..017bc6748cce 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java @@ -27,7 +27,7 @@ import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; -import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.ITaskExecutionRunnable; import java.util.Optional; @@ -48,13 +48,15 @@ public MasterTaskDispatcher(TaskEventService taskEventService, } @Override - protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { - TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + protected void doDispatch(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException { + TaskExecutionContext taskExecutionContext = + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext(); try { ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory .getProxyClient(taskExecutionContext.getHost(), ILogicTaskInstanceOperator.class); LogicTaskDispatchResponse logicTaskDispatchResponse = taskInstanceOperator - .dispatchLogicTask(new LogicTaskDispatchRequest(taskExecuteRunnable.getTaskExecutionContext())); + .dispatchLogicTask(new LogicTaskDispatchRequest( + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext())); if (!logicTaskDispatchResponse.isDispatchSuccess()) { throw new TaskDispatchException(String.format("Dispatch LogicTask to %s failed, response is: %s", taskExecutionContext.getHost(), logicTaskDispatchResponse)); @@ -68,7 +70,7 @@ protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDi } @Override - protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) { + protected Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable taskExecutionContext) { return masterTaskExecuteHost; } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java index f595d5a490f6..07c120d57286 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java @@ -19,13 +19,13 @@ import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; -import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.ITaskExecutionRunnable; /** * Used to do task dispatcher. */ public interface TaskDispatcher { - void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException, WorkerGroupNotFoundException; + void dispatchTask(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException, WorkerGroupNotFoundException; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java index 33200826947b..5faa4b745bdb 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java @@ -31,7 +31,7 @@ import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; -import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.ITaskExecutionRunnable; import java.util.Optional; @@ -53,13 +53,15 @@ public WorkerTaskDispatcher(TaskEventService taskEventService, } @Override - protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { - TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + protected void doDispatch(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException { + TaskExecutionContext taskExecutionContext = + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext(); try { ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory .getProxyClient(taskExecutionContext.getHost(), ITaskInstanceOperator.class); TaskInstanceDispatchResponse taskInstanceDispatchResponse = taskInstanceOperator - .dispatchTask(new TaskInstanceDispatchRequest(taskExecuteRunnable.getTaskExecutionContext())); + .dispatchTask(new TaskInstanceDispatchRequest( + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext())); if (!taskInstanceDispatchResponse.isDispatchSuccess()) { throw new TaskDispatchException(String.format("Dispatch task to %s failed, response is: %s", taskExecutionContext.getHost(), taskInstanceDispatchResponse)); @@ -73,8 +75,9 @@ protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDi } @Override - protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) throws WorkerGroupNotFoundException { - String workerGroup = taskExecuteRunnable.getTaskExecutionContext().getWorkerGroup(); + protected Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable ITaskExecutionRunnable) throws WorkerGroupNotFoundException { + String workerGroup = + ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext().getWorkerGroup(); return hostManager.select(workerGroup); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java index ab749b58615d..0bf5e2e4d31f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java @@ -18,13 +18,11 @@ package org.apache.dolphinscheduler.server.master.runner.execute; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; -import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnableFactory; -import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.TaskExecutionRunnableContext; +import org.apache.dolphinscheduler.server.master.runner.TaskExecutionRunnableContextFactory; import org.apache.dolphinscheduler.server.master.runner.operator.TaskExecuteRunnableOperatorManager; import lombok.extern.slf4j.Slf4j; @@ -32,31 +30,40 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; +/** + * The implementation of {@link TaskExecuteRunnableFactory} used to create {@link DefaultTaskExecutionRunnable}. + */ @Slf4j @Component -public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFactory { - - @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; +public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFactory { @Autowired - private TaskExecutionContextFactory taskExecutionContextFactory; + private TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; @Autowired - private TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; + private TaskExecutionRunnableContextFactory taskExecutionRunnableContextFactory; + /** + * Create DefaultTaskExecuteRunnable + * + * @param taskInstance task instance + * @return DefaultTaskExecuteRunnable + * @throws TaskExecuteRunnableCreateException If create DefaultTaskExecuteRunnable failed + */ @Override - public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException { - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId()); + public DefaultTaskExecutionRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException { + + if (taskInstance == null) { + throw new TaskExecuteRunnableCreateException("TaskInstance is null"); + } + try { - return new DefaultTaskExecuteRunnable( - workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(), - taskInstance, - taskExecutionContextFactory.createTaskExecutionContext(taskInstance), - taskExecuteRunnableOperatorManager); - } catch (TaskExecutionContextCreateException ex) { - throw new TaskExecuteRunnableCreateException("Create DefaultTaskExecuteRunnable failed", ex); + TaskExecutionRunnableContext taskExecutionRunnableContext = + taskExecutionRunnableContextFactory.createTaskExecutionRunnableContext(taskInstance); + return new DefaultTaskExecutionRunnable(taskExecutionRunnableContext, taskExecuteRunnableOperatorManager); + } catch (Exception ex) { + throw new TaskExecuteRunnableCreateException( + "Create DefaultTaskExecuteRunnable for taskInstance: " + taskInstance.getName() + " failed", ex); } } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableDispatchOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableDispatchOperator.java index cb2c7a0e07a9..cb2cf2c326b0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableDispatchOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableDispatchOperator.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueue; import java.util.concurrent.TimeUnit; @@ -42,9 +42,9 @@ public BaseTaskExecuteRunnableDispatchOperator( } @Override - public void operate(DefaultTaskExecuteRunnable taskExecuteRunnable) { + public void operate(DefaultTaskExecutionRunnable taskExecuteRunnable) { long remainTime = taskExecuteRunnable.getDelay(TimeUnit.SECONDS); - TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); + TaskInstance taskInstance = taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance(); if (remainTime > 0) { taskInstance.setState(TaskExecutionStatus.DELAY_EXECUTION); taskInstanceDao.updateById(taskInstance); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableKillOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableKillOperator.java index 1b7a92db98d2..8163f28fff95 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableKillOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableKillOperator.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import java.util.Date; @@ -36,8 +36,8 @@ public BaseTaskExecuteRunnableKillOperator(TaskInstanceDao taskInstanceDao) { } @Override - public void operate(DefaultTaskExecuteRunnable taskExecuteRunnable) { - TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); + public void operate(DefaultTaskExecutionRunnable taskExecuteRunnable) { + TaskInstance taskInstance = taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance(); log.info("Begin to kill task instance: {}", taskInstance.getName()); if (taskInstance.getState().isFinished()) { log.info("The task stance {} is finished, no need to kill", taskInstance.getName()); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnablePauseOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnablePauseOperator.java index 8163817afc66..81fd6a8e5f93 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnablePauseOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnablePauseOperator.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.server.master.runner.operator; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import lombok.extern.slf4j.Slf4j; @@ -26,9 +26,10 @@ public abstract class BaseTaskExecuteRunnablePauseOperator implements TaskExecuteRunnableOperator { @Override - public void operate(DefaultTaskExecuteRunnable taskExecuteRunnable) { + public void operate(DefaultTaskExecutionRunnable taskExecuteRunnable) { try { - pauseRemoteTaskInstanceInThreadPool(taskExecuteRunnable.getTaskInstance()); + pauseRemoteTaskInstanceInThreadPool( + taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance()); } catch (Exception e) { log.error("Pause DefaultTaskExecuteRunnable failed", e); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableTimeoutOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableTimeoutOperator.java index ef9dc8090119..f23443a07539 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableTimeoutOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/BaseTaskExecuteRunnableTimeoutOperator.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import java.util.Date; @@ -37,12 +37,12 @@ public BaseTaskExecuteRunnableTimeoutOperator(TaskInstanceDao taskInstanceDao) { } @Override - public void operate(DefaultTaskExecuteRunnable taskExecuteRunnable) { + public void operate(DefaultTaskExecutionRunnable taskExecuteRunnable) { // Right now, if the task is running in worker, the timeout strategy will be handled at worker side. // if the task is in master, the timeout strategy will be handled at master side. // todo: we should unify this, the master only need to handle the timeout strategy. and send request to worker // to kill the task, if the strategy is timeout_failed. - TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); + TaskInstance taskInstance = taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance(); TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); if (TaskTimeoutStrategy.FAILED != taskTimeoutStrategy && TaskTimeoutStrategy.WARNFAILED != taskTimeoutStrategy) { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperator.java index 1d397e3575a0..ec2dfc9f615c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperator.java @@ -17,10 +17,10 @@ package org.apache.dolphinscheduler.server.master.runner.operator; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; public interface TaskExecuteRunnableOperator { - void operate(DefaultTaskExecuteRunnable taskExecuteRunnable); + void operate(DefaultTaskExecutionRunnable taskExecuteRunnable); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperatorManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperatorManager.java index 1b92f5e75ca9..81ca49bc0f8c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperatorManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableOperatorManager.java @@ -17,7 +17,7 @@ package org.apache.dolphinscheduler.server.master.runner.operator; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.utils.TaskUtils; import org.springframework.beans.factory.annotation.Autowired; @@ -50,29 +50,33 @@ public class TaskExecuteRunnableOperatorManager { @Autowired private LogicTaskExecuteRunnableTimeoutOperator logicTaskTimeoutOperator; - public TaskExecuteRunnableOperator getTaskKillOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) { - if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) { + public TaskExecuteRunnableOperator getTaskKillOperator(DefaultTaskExecutionRunnable defaultTaskExecuteRunnable) { + if (TaskUtils.isMasterTask( + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getTaskType())) { return logicTaskKillOperator; } return taskKillOperator; } - public TaskExecuteRunnableOperator getTaskPauseOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) { - if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) { + public TaskExecuteRunnableOperator getTaskPauseOperator(DefaultTaskExecutionRunnable defaultTaskExecuteRunnable) { + if (TaskUtils.isMasterTask( + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getTaskType())) { return logicTaskPauseOperator; } return taskPauseOperator; } - public TaskExecuteRunnableOperator getTaskDispatchOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) { - if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) { + public TaskExecuteRunnableOperator getTaskDispatchOperator(DefaultTaskExecutionRunnable defaultTaskExecuteRunnable) { + if (TaskUtils.isMasterTask( + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getTaskType())) { return logicTaskDispatchOperator; } return taskDispatchOperator; } - public TaskExecuteRunnableOperator getTaskTimeoutOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) { - if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) { + public TaskExecuteRunnableOperator getTaskTimeoutOperator(DefaultTaskExecutionRunnable defaultTaskExecuteRunnable) { + if (TaskUtils.isMasterTask( + defaultTaskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance().getTaskType())) { return logicTaskTimeoutOperator; } return taskTimeoutOperator; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java index f450044377f2..fefd22c5f38a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java @@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseRequest; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseResponse; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; import org.apache.commons.lang3.StringUtils; @@ -35,9 +35,10 @@ public class TaskExecuteRunnablePauseOperator implements TaskExecuteRunnableOperator { @Override - public void operate(DefaultTaskExecuteRunnable taskExecuteRunnable) { + public void operate(DefaultTaskExecutionRunnable taskExecuteRunnable) { try { - pauseRemoteTaskInstanceInThreadPool(taskExecuteRunnable.getTaskInstance()); + pauseRemoteTaskInstanceInThreadPool( + taskExecuteRunnable.getTaskExecutionRunnableContext().getTaskInstance()); } catch (Exception e) { log.error("Pause DefaultTaskExecuteRunnable failed", e); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java index acc05aaf2d6c..52d93828f3f7 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java @@ -33,7 +33,7 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters; import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; @@ -52,20 +52,20 @@ public class BlockingLogicTask extends BaseSyncLogicTask { public static final String TASK_TYPE = "BLOCKING"; - private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private final IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; private final ProcessInstanceDao processInstanceDao; private final TaskInstanceDao taskInstanceDao; public BlockingLogicTask(TaskExecutionContext taskExecutionContext, - ProcessInstanceExecCacheManager processInstanceExecCacheManager, + IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository, ProcessInstanceDao processInstanceDao, TaskInstanceDao taskInstanceDao) { super(taskExecutionContext, JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() { })); - this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.IWorkflowExecuteRunnableRepository = IWorkflowExecuteRunnableRepository; this.processInstanceDao = processInstanceDao; this.taskInstanceDao = taskInstanceDao; } @@ -79,8 +79,8 @@ public void handle() throws MasterTaskExecuteException { : DependResult.FAILED; boolean isBlocked = (expected == conditionResult); log.info("blocking opportunity: expected-->{}, actual-->{}", expected, conditionResult); - ProcessInstance workflowInstance = processInstanceExecCacheManager - .getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()).getWorkflowExecuteContext() + ProcessInstance workflowInstance = IWorkflowExecuteRunnableRepository + .getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()).getWorkflowExecutionContext() .getWorkflowInstance(); workflowInstance.setBlocked(isBlocked); if (isBlocked) { @@ -99,9 +99,9 @@ private DependResult calculateConditionResult() throws MasterTaskExecuteExceptio // todo: we need to parse the task parameter from TaskExecutionContext TaskInstance taskInstance = - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) - .getTaskInstance(taskExecutionContext.getTaskInstanceId()) - .orElseThrow(() -> new MasterTaskExecuteException("Task instance not found")); + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) + .getTaskExecutionRunnableContext().getTaskInstance(); DependentParameters dependentParameters = taskInstance.getDependency(); List tempResultList = new ArrayList<>(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java index b4fdd56c100a..a818730aa348 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -36,11 +36,11 @@ public class BlockingLogicTaskPluginFactory implements ILogicTaskPluginFactory { private final ProcessInstanceDao workflowInstanceDao; public ConditionLogicTask(TaskExecutionContext taskExecutionContext, - ProcessInstanceExecCacheManager processInstanceExecCacheManager, + IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository, TaskInstanceDao taskInstanceDao, ProcessInstanceDao workflowInstanceDao) throws LogicTaskInitializeException { // todo: we need to change the parameter in front-end, so that we can directly use json to parse super(taskExecutionContext, - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) - .getTaskInstance(taskExecutionContext.getTaskInstanceId()) - .orElseThrow(() -> new LogicTaskInitializeException( - "Cannot find the task instance in workflow execute runnable")) + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) + .getTaskExecutionRunnableContext().getTaskInstance() .getDependency()); // todo:check the parameters, why we don't use conditionTask? taskInstance.getDependency(); this.taskInstanceDao = taskInstanceDao; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java index d6887df6b593..a963e3be38a2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; @@ -39,11 +39,11 @@ public class ConditionLogicTaskPluginFactory implements ILogicTaskPluginFactory< private ProcessInstanceDao processInstanceDao; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Override public ConditionLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { - return new ConditionLogicTask(taskExecutionContext, processInstanceExecCacheManager, taskInstanceDao, + return new ConditionLogicTask(taskExecutionContext, IWorkflowExecuteRunnableRepository, taskInstanceDao, processInstanceDao); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java index e6ee3a78ca7f..59332d79ac23 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java @@ -26,10 +26,10 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; @@ -48,7 +48,7 @@ public class DependentLogicTask extends BaseAsyncLogicTask private final TaskInstanceDao taskInstanceDao; private final ProcessInstanceDao processInstanceDao; - private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private final IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; public DependentLogicTask(TaskExecutionContext taskExecutionContext, ProjectDao projectDao, @@ -56,9 +56,9 @@ public DependentLogicTask(TaskExecutionContext taskExecutionContext, TaskDefinitionDao taskDefinitionDao, TaskInstanceDao taskInstanceDao, ProcessInstanceDao processInstanceDao, - ProcessInstanceExecCacheManager processInstanceExecCacheManager) throws LogicTaskInitializeException { + IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository) throws LogicTaskInitializeException { super(taskExecutionContext, - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) .getTaskInstance(taskExecutionContext.getTaskInstanceId()) .orElseThrow(() -> new LogicTaskInitializeException( "Cannot find the task instance in workflow execute runnable")) @@ -68,7 +68,7 @@ public DependentLogicTask(TaskExecutionContext taskExecutionContext, this.taskDefinitionDao = taskDefinitionDao; this.taskInstanceDao = taskInstanceDao; this.processInstanceDao = processInstanceDao; - this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.IWorkflowExecuteRunnableRepository = IWorkflowExecuteRunnableRepository; } @@ -85,8 +85,8 @@ public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() { @Override public void pause() throws MasterTaskExecuteException { - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); if (workflowExecuteRunnable == null) { log.error("Cannot find the WorkflowExecuteRunnable"); return; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java index e8012ac3f9b1..295e925d46ce 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; @@ -48,7 +48,7 @@ public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory< private ProcessInstanceDao processInstanceDao; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Override public DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { @@ -59,7 +59,7 @@ public DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionCont taskDefinitionDao, taskInstanceDao, processInstanceDao, - processInstanceExecCacheManager); + IWorkflowExecuteRunnableRepository); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java index 1883a27d8b85..4354f465daa5 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java @@ -27,9 +27,9 @@ import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStateChangeEvent; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; @@ -41,16 +41,16 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask { public static final String TASK_TYPE = "SUB_PROCESS"; - private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private final IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; private final ProcessInstanceDao processInstanceDao; public SubWorkflowLogicTask(TaskExecutionContext taskExecutionContext, - ProcessInstanceExecCacheManager processInstanceExecCacheManager, + IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository, ProcessInstanceDao processInstanceDao) { super(taskExecutionContext, JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() { })); - this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.IWorkflowExecuteRunnableRepository = IWorkflowExecuteRunnableRepository; this.processInstanceDao = processInstanceDao; } @@ -62,8 +62,8 @@ public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskE @Override public void pause() throws MasterTaskExecuteException { - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); if (workflowExecuteRunnable == null) { log.warn("Cannot find WorkflowExecuteRunnable"); return; @@ -99,8 +99,8 @@ public void pause() throws MasterTaskExecuteException { @Override public void kill() { - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); if (workflowExecuteRunnable == null) { log.warn("Cannot find WorkflowExecuteRunnable"); return; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java index ef32515b2d2c..e3660bc1c135 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java @@ -19,7 +19,7 @@ import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; import lombok.extern.slf4j.Slf4j; @@ -34,11 +34,11 @@ public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactor @Autowired private ProcessInstanceDao processInstanceDao; @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Override public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) { - return new SubWorkflowLogicTask(taskExecutionContext, processInstanceExecCacheManager, processInstanceDao); + return new SubWorkflowLogicTask(taskExecutionContext, IWorkflowExecuteRunnableRepository, processInstanceDao); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java index 244926c09613..46c54ea56aca 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java @@ -26,10 +26,10 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property; import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo; import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; import org.apache.dolphinscheduler.server.master.utils.SwitchTaskUtils; @@ -51,20 +51,19 @@ public class SwitchLogicTask extends BaseSyncLogicTask { private final TaskInstance taskInstance; public SwitchLogicTask(TaskExecutionContext taskExecutionContext, - ProcessInstanceExecCacheManager processInstanceExecCacheManager) throws LogicTaskInitializeException { + IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository) throws LogicTaskInitializeException { super(taskExecutionContext, // todo: we need to refactor the logic task parameter........ - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) - .getTaskInstance(taskExecutionContext.getTaskInstanceId()) - .orElseThrow(() -> new LogicTaskInitializeException( - "Cannot find the task instance in workflow execute runnable")) + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) + .getTaskExecutionRunnableContext().getTaskInstance() .getSwitchDependency()); - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); - this.processInstance = workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(); - this.taskInstance = workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()) - .orElseThrow(() -> new LogicTaskInitializeException( - "Cannot find the task instance in workflow execute runnable")); + IWorkflowExecutionRunnable workflowExecuteRunnable = + IWorkflowExecuteRunnableRepository.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + this.processInstance = workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + this.taskInstance = + workflowExecuteRunnable.getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) + .getTaskExecutionRunnableContext().getTaskInstance(); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java index dfb47aa92d75..50b0accede9d 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.server.master.runner.task.switchtask; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; @@ -32,11 +32,11 @@ public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Override public SwitchLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { - return new SwitchLogicTask(taskExecutionContext, processInstanceExecCacheManager); + return new SwitchLogicTask(taskExecutionContext, IWorkflowExecuteRunnableRepository); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/ExecutingService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/ExecutingService.java index 2120e41ac3e3..32d454f3e7f8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/ExecutingService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/ExecutingService.java @@ -20,8 +20,8 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.extract.master.dto.TaskInstanceExecuteDto; import org.apache.dolphinscheduler.extract.master.dto.WorkflowExecuteDto; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.collections4.CollectionUtils; @@ -44,11 +44,11 @@ public class ExecutingService { @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; public Optional queryWorkflowExecutingData(Integer processInstanceId) { - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = null; + // IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId); if (workflowExecuteRunnable == null) { log.info("workflow execute data not found, maybe it has finished, workflow id:{}", processInstanceId); return Optional.empty(); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/MasterFailoverService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/MasterFailoverService.java index 09be610cba72..8dc961e122ae 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/MasterFailoverService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/MasterFailoverService.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -54,17 +54,17 @@ public class MasterFailoverService { private final ProcessService processService; private final String localAddress; - private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private final IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; public MasterFailoverService(@NonNull RegistryClient registryClient, @NonNull MasterConfig masterConfig, @NonNull ProcessService processService, - @NonNull ProcessInstanceExecCacheManager processInstanceExecCacheManager) { + @NonNull IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository) { this.registryClient = registryClient; this.masterConfig = masterConfig; this.processService = processService; this.localAddress = masterConfig.getMasterAddress(); - this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.IWorkflowExecuteRunnableRepository = IWorkflowExecuteRunnableRepository; } /** @@ -192,7 +192,7 @@ private boolean checkProcessInstanceNeedFailover(Optional beFailoveredMast return false; } - if (processInstanceExecCacheManager.contains(processInstance.getId())) { + if (IWorkflowExecuteRunnableRepository.contains(processInstance.getId())) { // the processInstance is a running process instance in the current master log.info("The workflowInstance is running in the current master, no need to failover"); return false; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java index 86515d032d84..7fa111793f90 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java @@ -34,11 +34,11 @@ import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import org.apache.dolphinscheduler.server.master.utils.TaskUtils; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -71,7 +71,7 @@ public class WorkerFailoverService { private final MasterConfig masterConfig; private final ProcessService processService; private final WorkflowExecuteThreadPool workflowExecuteThreadPool; - private final ProcessInstanceExecCacheManager cacheManager; + private final IWorkflowExecuteRunnableRepository cacheManager; private final String localAddress; private final TaskInstanceDao taskInstanceDao; @@ -80,7 +80,7 @@ public WorkerFailoverService(@NonNull RegistryClient registryClient, @NonNull MasterConfig masterConfig, @NonNull ProcessService processService, @NonNull WorkflowExecuteThreadPool workflowExecuteThreadPool, - @NonNull ProcessInstanceExecCacheManager cacheManager, + @NonNull IWorkflowExecuteRunnableRepository cacheManager, @NonNull TaskInstanceDao taskInstanceDao) { this.registryClient = registryClient; this.masterConfig = masterConfig; @@ -123,8 +123,9 @@ public void failoverWorker(@NonNull String workerHost) { LogUtils.setWorkflowAndTaskInstanceIDMDC(taskInstance.getProcessInstanceId(), taskInstance.getId()); ProcessInstance processInstance = processInstanceCacheMap.computeIfAbsent( taskInstance.getProcessInstanceId(), k -> { - WorkflowExecuteRunnable workflowExecuteRunnable = cacheManager.getByProcessInstanceId( - taskInstance.getProcessInstanceId()); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + cacheManager.getByProcessInstanceId( + taskInstance.getProcessInstanceId()); if (workflowExecuteRunnable == null) { return null; } 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 new file mode 100644 index 000000000000..3f2a93755781 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -0,0 +1,14 @@ +package org.apache.dolphinscheduler.server.master.utils; + +import lombok.experimental.UtilityClass; + +import org.springframework.dao.DataAccessResourceFailureException; + +@UtilityClass +public class ExceptionUtils { + + public boolean isDatabaseConnectedFailedException(Throwable e) { + return e instanceof DataAccessResourceFailureException; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngine.java new file mode 100644 index 000000000000..8f499ffc8491 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngine.java @@ -0,0 +1,73 @@ +package org.apache.dolphinscheduler.server.master.workflow; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DefaultWorkflowEngine implements IWorkflowEngine { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + public DefaultWorkflowEngine(IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) { + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + } + + @Override + public void start() { + log.info("{} started", getClass().getName()); + } + + @Override + public void triggerWorkflow(IWorkflowExecutionRunnable workflowExecuteRunnable) { + ProcessInstance workflowInstance = workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + try { + log.info("Begin to trigger WorkflowExecutionRunnable: {}", workflowInstance.getName()); + workflowExecuteRunnableRepository.cache(workflowInstance.getId(), workflowExecuteRunnable); + // todo: create WOrkflowExecutionTriggerEvent + workflowExecuteRunnable.start(); + } catch (Throwable throwable) { + workflowExecuteRunnableRepository.removeByProcessInstanceId(workflowInstance.getId()); + throw throwable; + } + } + + @Override + public void pauseWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Pausing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstance().getName()); + workflowExecuteRunnable.pause(); + } + + @Override + public void killWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Killing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstance().getName()); + workflowExecuteRunnable.kill(); + } + + @Override + public void finalizeWorkflow(Integer workflowInstanceId) { + log.info("Finalizing WorkflowExecutionRunnable: {}", workflowInstanceId); + workflowExecuteRunnableRepository.removeByProcessInstanceId(workflowInstanceId); + } + + @Override + public void stop() { + log.info("{} stopped", getClass().getName()); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/IWorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/IWorkflowEngine.java new file mode 100644 index 000000000000..7587d6f12a0e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/workflow/IWorkflowEngine.java @@ -0,0 +1,50 @@ +package org.apache.dolphinscheduler.server.master.workflow; + +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +/** + * The WorkflowEngine is responsible for starting, stopping, pausing, and finalizing workflows. + */ +public interface IWorkflowEngine { + + /** + * Start the workflow engine. + */ + void start(); + + /** + * Trigger a workflow to start. + * + * @param workflowExecuteRunnable the workflow to start + */ + void triggerWorkflow(IWorkflowExecutionRunnable workflowExecuteRunnable); + + /** + * Pause a workflow instance. + * + * @param workflowInstanceId the ID of the workflow to pause + * @throws WorkflowExecuteRunnableNotFoundException if the workflow is not found + */ + void pauseWorkflow(Integer workflowInstanceId); + + /** + * Kill a workflow instance. + * + * @param workflowInstanceId the ID of the workflow to stop + * @throws WorkflowExecuteRunnableNotFoundException if the workflow is not found + */ + void killWorkflow(Integer workflowInstanceId); + + /** + * Finalize a workflow instance. + * + * @param workflowInstanceId the ID of the workflow to finalize + */ + void finalizeWorkflow(Integer workflowInstanceId); + + /** + * Stop the workflow engine. + */ + void stop(); +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImplTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImplTest.java similarity index 73% rename from dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImplTest.java rename to dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImplTest.java index 05b8f7aa13a8..b6b3c5deb9c1 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImplTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cache/impl/WorkflowExecuteRunnableRepositoryImplTest.java @@ -17,7 +17,7 @@ package org.apache.dolphinscheduler.server.master.cache.impl; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; import java.util.Collection; @@ -33,13 +33,13 @@ @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) -public class ProcessInstanceExecCacheManagerImplTest { +public class WorkflowExecuteRunnableRepositoryImplTest { @InjectMocks - private ProcessInstanceExecCacheManagerImpl processInstanceExecCacheManager; + private WorkflowExecuteRunnableRepositoryImpl processInstanceExecCacheManager; @Mock - private WorkflowExecuteRunnable workflowExecuteThread; + private IWorkflowExecutionRunnable workflowExecuteThread; @BeforeEach public void before() { @@ -48,7 +48,8 @@ public void before() { @Test public void testGetByProcessInstanceId() { - WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(1); + IWorkflowExecutionRunnable workflowExecuteThread = + processInstanceExecCacheManager.getByProcessInstanceId(1); Assertions.assertNotNull(workflowExecuteThread); } @@ -60,20 +61,22 @@ public void testContains() { @Test public void testCacheNull() { Assertions.assertThrows(NullPointerException.class, () -> processInstanceExecCacheManager.cache(2, null)); - WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(2); + IWorkflowExecutionRunnable workflowExecuteThread = + processInstanceExecCacheManager.getByProcessInstanceId(2); Assertions.assertNull(workflowExecuteThread); } @Test public void testRemoveByProcessInstanceId() { processInstanceExecCacheManager.removeByProcessInstanceId(1); - WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(1); + IWorkflowExecutionRunnable workflowExecuteThread = + processInstanceExecCacheManager.getByProcessInstanceId(1); Assertions.assertNull(workflowExecuteThread); } @Test public void testGetAll() { - Collection workflowExecuteThreads = processInstanceExecCacheManager.getAll(); + Collection workflowExecuteThreads = processInstanceExecCacheManager.getAll(); Assertions.assertEquals(1, workflowExecuteThreads.size()); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandlerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandlerTest.java index 4e1af4dc27a8..89522f2feaab 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandlerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/event/TaskCacheEventHandlerTest.java @@ -23,9 +23,9 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import org.apache.dolphinscheduler.server.master.utils.DataQualityResultOperator; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -49,7 +49,7 @@ class TaskCacheEventHandlerTest { private TaskCacheEventHandler taskCacheEventHandler; @Mock - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; @Mock private WorkflowExecuteThreadPool workflowExecuteThreadPool; @@ -82,8 +82,8 @@ void testHandleTaskEvent() { Mockito.when(taskInstanceDao.queryById(cacheTaskInstanceId)).thenReturn(cacheTaskInstance); - WorkflowExecuteRunnable workflowExecuteRunnable = Mockito.mock(WorkflowExecuteRunnable.class); - Mockito.when(processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId)) + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = Mockito.mock(DefaultWorkflowExecutionRunnable.class); + Mockito.when(IWorkflowExecuteRunnableRepository.getByProcessInstanceId(processInstanceId)) .thenReturn(workflowExecuteRunnable); Optional taskInstanceOptional = Mockito.mock(Optional.class); Mockito.when(workflowExecuteRunnable.getTaskInstance(taskInstanceId)).thenReturn(taskInstanceOptional); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/EventEngineTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/EventEngineTest.java new file mode 100644 index 000000000000..3ff85fd681d9 --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/EventEngineTest.java @@ -0,0 +1,7 @@ +package org.apache.dolphinscheduler.server.master.events; + +class EventEngineTest { + + private EventEngine eventEngine; + +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java index c08fb206f4dc..04234ad6ba3c 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java @@ -77,7 +77,7 @@ @MockitoSettings(strictness = Strictness.LENIENT) public class WorkflowExecuteRunnableTest { - private WorkflowExecuteRunnable workflowExecuteThread; + private DefaultWorkflowExecutionRunnable workflowExecuteThread; private ProcessInstance processInstance; @@ -134,7 +134,7 @@ public void init() throws Exception { stateWheelExecuteThread = Mockito.mock(StateWheelExecuteThread.class); curingGlobalParamsService = Mockito.mock(CuringParamsService.class); ProcessAlertManager processAlertManager = Mockito.mock(ProcessAlertManager.class); - WorkflowExecuteContext workflowExecuteContext = Mockito.mock(WorkflowExecuteContext.class); + WorkflowExecutionContext workflowExecuteContext = Mockito.mock(WorkflowExecutionContext.class); Mockito.when(workflowExecuteContext.getWorkflowInstance()).thenReturn(processInstance); IWorkflowGraph workflowGraph = Mockito.mock(IWorkflowGraph.class); Mockito.when(workflowExecuteContext.getWorkflowGraph()).thenReturn(workflowGraph); @@ -143,7 +143,7 @@ public void init() throws Exception { taskGroupCoordinator = Mockito.mock(TaskGroupCoordinator.class); workflowExecuteThread = Mockito.spy( - new WorkflowExecuteRunnable( + new DefaultWorkflowExecutionRunnable( workflowExecuteContext, commandService, processService, @@ -163,7 +163,7 @@ public void testParseStartNodeName() { try { Map cmdParam = new HashMap<>(); cmdParam.put(CMD_PARAM_START_NODES, "1,2,3"); - Class masterExecThreadClass = WorkflowExecuteRunnable.class; + Class masterExecThreadClass = DefaultWorkflowExecutionRunnable.class; Method method = masterExecThreadClass.getDeclaredMethod("parseStartNodeName", String.class); method.setAccessible(true); List nodeNames = @@ -191,7 +191,7 @@ public void testGetStartTaskInstanceList() { Arrays.asList(taskInstance1.getId(), taskInstance2.getId(), taskInstance3.getId(), taskInstance4.getId()))) .thenReturn(Arrays.asList(taskInstance1, taskInstance2, taskInstance3, taskInstance4)); - Class masterExecThreadClass = WorkflowExecuteRunnable.class; + Class masterExecThreadClass = DefaultWorkflowExecutionRunnable.class; Method method = masterExecThreadClass.getDeclaredMethod("getRecoverTaskInstanceList", String.class); method.setAccessible(true); List taskInstances = @@ -241,7 +241,7 @@ public void testGetPreVarPool() { completeTaskSet.add(taskInstance1.getTaskCode()); completeTaskSet.add(taskInstance2.getTaskCode()); - Class masterExecThreadClass = WorkflowExecuteRunnable.class; + Class masterExecThreadClass = DefaultWorkflowExecutionRunnable.class; Field completeTaskSetField = masterExecThreadClass.getDeclaredField("completeTaskSet"); completeTaskSetField.setAccessible(true); @@ -321,7 +321,7 @@ public void testClearDataIfExecuteTask() throws NoSuchFieldException, IllegalAcc completeTaskSet.add(taskInstance1.getTaskCode()); completeTaskSet.add(taskInstance2.getTaskCode()); - Class masterExecThreadClass = WorkflowExecuteRunnable.class; + Class masterExecThreadClass = DefaultWorkflowExecutionRunnable.class; Field completeTaskMapField = masterExecThreadClass.getDeclaredField("completeTaskSet"); completeTaskMapField.setAccessible(true); @@ -346,7 +346,7 @@ public void testClearDataIfExecuteTask() throws NoSuchFieldException, IllegalAcc Mockito.when(dag.containsNode(1L)).thenReturn(true); Mockito.when(dag.containsNode(2L)).thenReturn(false); - WorkflowExecuteContext workflowExecuteContext = Mockito.mock(WorkflowExecuteContext.class); + WorkflowExecutionContext workflowExecuteContext = Mockito.mock(WorkflowExecutionContext.class); Mockito.when(workflowExecuteContext.getWorkflowInstance()).thenReturn(processInstance); IWorkflowGraph workflowGraph = Mockito.mock(IWorkflowGraph.class); Mockito.when(workflowExecuteContext.getWorkflowGraph()).thenReturn(workflowGraph); @@ -388,14 +388,14 @@ void testTryToDispatchTaskInstance() { TaskInstance taskInstance = new TaskInstance(); taskInstance.setState(TaskExecutionStatus.PAUSE); Mockito.when(processInstance.isBlocked()).thenReturn(true); - TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class); - workflowExecuteThread.tryToDispatchTaskInstance(taskInstance, taskExecuteRunnable); - Mockito.verify(taskExecuteRunnable, Mockito.never()).dispatch(); + ITaskExecutionRunnable ITaskExecutionRunnable = Mockito.mock(ITaskExecutionRunnable.class); + workflowExecuteThread.tryToDispatchTaskInstance(taskInstance, ITaskExecutionRunnable); + Mockito.verify(ITaskExecutionRunnable, Mockito.never()).dispatch(); // submit success should dispatch taskInstance = new TaskInstance(); taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); - workflowExecuteThread.tryToDispatchTaskInstance(taskInstance, taskExecuteRunnable); - Mockito.verify(taskExecuteRunnable).dispatch(); + workflowExecuteThread.tryToDispatchTaskInstance(taskInstance, ITaskExecutionRunnable); + Mockito.verify(ITaskExecutionRunnable).dispatch(); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java index f57c9b6a68dd..c278640df09b 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; -import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.ITaskExecutionRunnable; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -36,11 +36,11 @@ public void getTaskExecuteHost() { TaskEventService taskEventService = Mockito.mock(TaskEventService.class); MasterConfig masterConfig = Mockito.mock(MasterConfig.class); Mockito.when(masterConfig.getMasterAddress()).thenReturn("localhost:5678"); - TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class); + ITaskExecutionRunnable ITaskExecutionRunnable = Mockito.mock(ITaskExecutionRunnable.class); MasterTaskDispatcher masterTaskDispatcher = new MasterTaskDispatcher(taskEventService, masterConfig); - Host taskInstanceDispatchHost = masterTaskDispatcher.getTaskInstanceDispatchHost(taskExecuteRunnable) + Host taskInstanceDispatchHost = masterTaskDispatcher.getTaskInstanceDispatchHost(ITaskExecutionRunnable) .orElseThrow(() -> new IllegalArgumentException("Cannot get the ")); Assertions.assertEquals(masterConfig.getMasterAddress(), taskInstanceDispatchHost.getAddress()); } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java index 46c4f53e1a3e..3cb7e1eb4797 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; -import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.ITaskExecutionRunnable; import java.util.Optional; @@ -45,9 +45,12 @@ public void getTaskInstanceDispatchHost() throws WorkerGroupNotFoundException { WorkerTaskDispatcher workerTaskDispatcher = new WorkerTaskDispatcher(taskEventService, masterConfig, hostManager); - TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class); - Mockito.when(taskExecuteRunnable.getTaskExecutionContext()).thenReturn(new TaskExecutionContext()); - Optional taskInstanceDispatchHost = workerTaskDispatcher.getTaskInstanceDispatchHost(taskExecuteRunnable); + ITaskExecutionRunnable ITaskExecutionRunnable = + Mockito.mock(ITaskExecutionRunnable.class, Mockito.RETURNS_DEEP_STUBS); + Mockito.when(ITaskExecutionRunnable.getTaskExecutionRunnableContext().getTaskExecutionContext()) + .thenReturn(new TaskExecutionContext()); + Optional taskInstanceDispatchHost = + workerTaskDispatcher.getTaskInstanceDispatchHost(ITaskExecutionRunnable); Assertions.assertEquals("localhost:1234", taskInstanceDispatchHost.get().getAddress()); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactoryTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactoryTest.java new file mode 100644 index 000000000000..27bef81336b7 --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactoryTest.java @@ -0,0 +1,57 @@ +package org.apache.dolphinscheduler.server.master.runner.execute; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.TaskExecutionRunnableContext; +import org.apache.dolphinscheduler.server.master.runner.TaskExecutionRunnableContextFactory; +import org.apache.dolphinscheduler.server.master.runner.operator.TaskExecuteRunnableOperatorManager; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +class DefaultTaskExecuteRunnableFactoryTest { + + @InjectMocks + private DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; + + @Mock + private TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; + + @Mock + private TaskExecutionRunnableContextFactory taskExecutionRunnableContextFactory; + + @Test + void createTaskExecuteRunnable() throws TaskExecuteRunnableCreateException { + TaskExecuteRunnableCreateException taskExecuteRunnableCreateException = + assertThrows(TaskExecuteRunnableCreateException.class, + () -> defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(null)); + assertEquals("TaskInstance is null", taskExecuteRunnableCreateException.getMessage()); + + TaskInstance taskInstance = new TaskInstance(); + taskInstance.setName("test"); + doThrow(new NullPointerException("ProcessInstance is null")).when(taskExecutionRunnableContextFactory) + .createTaskExecutionRunnableContext(taskInstance); + taskExecuteRunnableCreateException = + assertThrows(TaskExecuteRunnableCreateException.class, + () -> defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance)); + assertEquals("Create DefaultTaskExecuteRunnable for taskInstance: test failed", + taskExecuteRunnableCreateException.getMessage()); + + doReturn(new TaskExecutionRunnableContext()).when(taskExecutionRunnableContextFactory) + .createTaskExecutionRunnableContext(taskInstance); + DefaultTaskExecutionRunnable taskExecuteRunnable = + defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance); + assertNotNull(taskExecuteRunnable); + } +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayTaskExecuteRunnableTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayITaskExecutionRunnableTest.java similarity index 70% rename from dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayTaskExecuteRunnableTest.java rename to dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayITaskExecutionRunnableTest.java index 778884e066a2..9b2bb7e55ea1 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayTaskExecuteRunnableTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityDelayITaskExecutionRunnableTest.java @@ -21,14 +21,15 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.PriorityDelayTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.PriorityDelayITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.TaskExecutionRunnableContext; import org.apache.dolphinscheduler.server.master.runner.operator.TaskExecuteRunnableOperatorManager; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class PriorityDelayTaskExecuteRunnableTest { +public class PriorityDelayITaskExecutionRunnableTest { @Test public void testCompareTo() { @@ -48,10 +49,22 @@ public void testCompareTo() { TaskExecutionContext context1 = new TaskExecutionContext(); TaskExecutionContext context2 = new TaskExecutionContext(); - PriorityDelayTaskExecuteRunnable p1 = - new DefaultTaskExecuteRunnable(workflowInstance, t1, context1, taskOperatorManager); - PriorityDelayTaskExecuteRunnable p2 = - new DefaultTaskExecuteRunnable(workflowInstance, t2, context2, taskOperatorManager); + + TaskExecutionRunnableContext taskExecutionRunnableContext1 = TaskExecutionRunnableContext.builder() + .workflowInstance(workflowInstance) + .taskInstance(t1) + .taskExecutionContext(context1) + .build(); + + TaskExecutionRunnableContext taskExecutionRunnableContext2 = TaskExecutionRunnableContext.builder() + .workflowInstance(workflowInstance) + .taskInstance(t2) + .taskExecutionContext(context2) + .build(); + PriorityDelayITaskExecutionRunnable p1 = + new DefaultTaskExecutionRunnable(taskExecutionRunnableContext1, taskOperatorManager); + PriorityDelayITaskExecutionRunnable p2 = + new DefaultTaskExecutionRunnable(taskExecutionRunnableContext2, taskOperatorManager); Assertions.assertEquals(0, p1.compareTo(p2)); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java index 2081d2dd7ad5..336cde9ae191 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java @@ -33,11 +33,11 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.StateEvent; -import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionContext; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -81,10 +81,10 @@ public class FailoverServiceTest { private WorkflowExecuteThreadPool workflowExecuteThreadPool; @Mock - private ProcessInstanceExecCacheManager cacheManager; + private IWorkflowExecuteRunnableRepository cacheManager; @Mock - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private IWorkflowExecuteRunnableRepository IWorkflowExecuteRunnableRepository; private static int masterPort = 5678; private static int workerPort = 1234; @@ -106,7 +106,7 @@ public void before() throws Exception { given(masterConfig.getMasterAddress()).willReturn(testMasterHost); MasterFailoverService masterFailoverService = new MasterFailoverService(registryClient, masterConfig, processService, - processInstanceExecCacheManager); + IWorkflowExecuteRunnableRepository); WorkerFailoverService workerFailoverService = new WorkerFailoverService(registryClient, masterConfig, processService, @@ -193,10 +193,10 @@ public void failoverMasterTest() { @Test public void failoverWorkTest() { workerTaskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - WorkflowExecuteRunnable workflowExecuteRunnable = Mockito.mock(WorkflowExecuteRunnable.class); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = Mockito.mock(DefaultWorkflowExecutionRunnable.class); Mockito.when(workflowExecuteRunnable.getAllTaskInstances()).thenReturn(Lists.newArrayList(workerTaskInstance)); - IWorkflowExecuteContext workflowExecuteRunnableContext = Mockito.mock(IWorkflowExecuteContext.class); + IWorkflowExecutionContext workflowExecuteRunnableContext = Mockito.mock(IWorkflowExecutionContext.class); Mockito.when(workflowExecuteRunnable.getWorkflowExecuteContext()).thenReturn(workflowExecuteRunnableContext); Mockito.when(workflowExecuteRunnableContext.getWorkflowInstance()).thenReturn(processInstance); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngineTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngineTest.java new file mode 100644 index 000000000000..889535029f7c --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/workflow/DefaultWorkflowEngineTest.java @@ -0,0 +1,94 @@ +package org.apache.dolphinscheduler.server.master.workflow; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.verify; + +import org.apache.dolphinscheduler.server.master.cache.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.cache.impl.WorkflowExecuteRunnableRepositoryImpl; +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.server.master.runner.DefaultWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecutionRunnable; + +import org.apache.commons.lang3.RandomUtils; + +import org.junit.jupiter.api.Test; +import org.mockito.Answers; +import org.mockito.Mockito; + +class DefaultWorkflowEngineTest { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository = + new WorkflowExecuteRunnableRepositoryImpl(); + + private final DefaultWorkflowEngine defaultWorkflowEngine = + new DefaultWorkflowEngine(workflowExecuteRunnableRepository); + + @Test + void start() { + defaultWorkflowEngine.start(); + } + + @Test + void triggerWorkflow() { + int workflowInstanceId = RandomUtils.nextInt(0, 100); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = createWorkflowExecuteRunnable(workflowInstanceId); + defaultWorkflowEngine.triggerWorkflow(workflowExecuteRunnable); + assertEquals(workflowExecuteRunnable, + workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId)); + workflowExecuteRunnableRepository.removeByProcessInstanceId(workflowInstanceId); + + doThrow(new RuntimeException()).when(workflowExecuteRunnable).start(); + assertThrows(RuntimeException.class, () -> defaultWorkflowEngine.triggerWorkflow(workflowExecuteRunnable)); + assertNull(workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId)); + } + + @Test + void pauseWorkflow() { + int workflowInstanceId = RandomUtils.nextInt(0, 100); + assertThrows(WorkflowExecuteRunnableNotFoundException.class, + () -> defaultWorkflowEngine.pauseWorkflow(workflowInstanceId)); + + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = createWorkflowExecuteRunnable(workflowInstanceId); + workflowExecuteRunnableRepository.cache(workflowInstanceId, workflowExecuteRunnable); + defaultWorkflowEngine.pauseWorkflow(workflowInstanceId); + verify(workflowExecuteRunnable).pause(); + } + + @Test + void stopWorkflow() { + int workflowInstanceId = RandomUtils.nextInt(0, 100); + assertThrows(WorkflowExecuteRunnableNotFoundException.class, + () -> defaultWorkflowEngine.pauseWorkflow(workflowInstanceId)); + + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = createWorkflowExecuteRunnable(workflowInstanceId); + workflowExecuteRunnableRepository.cache(workflowInstanceId, workflowExecuteRunnable); + defaultWorkflowEngine.killWorkflow(workflowInstanceId); + verify(workflowExecuteRunnable).kill(); + } + + @Test + void finalizeWorkflow() { + int workflowInstanceId = RandomUtils.nextInt(0, 100); + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = createWorkflowExecuteRunnable(workflowInstanceId); + workflowExecuteRunnableRepository.cache(workflowInstanceId, workflowExecuteRunnable); + defaultWorkflowEngine.finalizeWorkflow(workflowInstanceId); + assertNull(workflowExecuteRunnableRepository.getByProcessInstanceId(workflowInstanceId)); + } + + @Test + void stop() { + defaultWorkflowEngine.stop(); + } + + private DefaultWorkflowExecutionRunnable createWorkflowExecuteRunnable(Integer workflowInstanceId) { + DefaultWorkflowExecutionRunnable workflowExecuteRunnable = + Mockito.mock(DefaultWorkflowExecutionRunnable.class, Answers.RETURNS_DEEP_STUBS); + Mockito.when(workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance().getId()) + .thenReturn(workflowInstanceId); + return workflowExecuteRunnable; + } + +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/alert/ListenerEventAlertManager.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/alert/ListenerEventAlertManager.java index 182ed678a0c2..f68932697de7 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/alert/ListenerEventAlertManager.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/alert/ListenerEventAlertManager.java @@ -122,7 +122,7 @@ public void publishProcessStartListenerEvent(ProcessInstance processInstance, Pr ProcessStartListenerEvent event = new ProcessStartListenerEvent(); event.setProjectCode(projectUser.getProjectCode()); event.setProjectName(projectUser.getProjectName()); - event.setOwner(projectUser.getUserName()); + event.setOwner(processInstance.getExecutorName()); event.setProcessId(processInstance.getId()); event.setProcessDefinitionCode(processInstance.getProcessDefinitionCode()); event.setProcessName(processInstance.getName()); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/exceptions/CronParseException.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/exceptions/CronParseException.java index cfc46d7572f4..d76e22084219 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/exceptions/CronParseException.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/exceptions/CronParseException.java @@ -17,7 +17,7 @@ package org.apache.dolphinscheduler.service.exceptions; -public class CronParseException extends Exception { +public class CronParseException extends RuntimeException { public CronParseException(String message) { super(message);