From 514d713aef5e84e9b1191052ec8b5d8ecbee8dad Mon Sep 17 00:00:00 2001 From: Wenjun Ruan Date: Tue, 5 Mar 2024 15:48:00 +0800 Subject: [PATCH] Add dolphinscheduler-workflow-engine module --- .../dao/entity/ProcessInstance.java | 8 +- .../repository/ProcessTaskRelationLogDao.java | 3 + .../impl/ProcessTaskRelationLogDaoImpl.java | 6 + .../server/master/dag/BasicDAG.java | 71 +++++ .../server/master/dag/DAG.java | 146 +++++++++ .../server/master/dag/DAGEdge.java | 51 ++++ .../server/master/dag/DAGEngine.java | 148 +++++++++ .../server/master/dag/DAGEngineFactory.java | 32 ++ .../server/master/dag/DAGNode.java | 92 ++++++ .../server/master/dag/IDAGEngine.java | 70 +++++ .../server/master/dag/IDAGEngineFactory.java | 24 ++ .../server/master/dag/IDAGNodeAction.java | 35 +++ .../IEventRepositoryFactory.java} | 10 +- .../dag/IEventfulExecutionRunnable.java | 34 +++ .../master/dag/ITaskExecutionContext.java | 26 ++ .../dag/ITaskExecutionContextFactory.java | 24 ++ .../master/dag/ITaskExecutionRunnable.java | 59 ++++ .../dag/ITaskExecutionRunnableFactory.java | 24 ++ .../dag/ITaskExecutionRunnableRepository.java | 33 ++ ...askExecutionRunnableRepositoryFactory.java | 24 ++ .../server/master/dag/IWorkflowDAG.java | 25 ++ .../master/dag/IWorkflowDAGFactory.java | 35 +++ .../server/master/dag/IWorkflowEngine.java | 57 ++++ .../master/dag/IWorkflowExecutionContext.java | 46 +++ .../master/dag/IWorkflowExecutionDAG.java | 74 +++++ .../dag/IWorkflowExecutionDAGFactory.java | 23 ++ .../dag/IWorkflowExecutionRunnable.java | 54 ++++ .../dag/MemoryEventRepositoryFactory.java | 35 +++ .../master/dag/TaskExecutionContext.java | 35 +++ .../master/dag/TaskExecutionRunnable.java | 56 ++++ .../dag/TaskExecutionRunnableRepository.java | 68 +++++ ...askExecutionRunnableRepositoryFactory.java | 32 ++ .../server/master/dag/TaskExecutionValve.java | 27 ++ .../server/master/dag/TaskIdentify.java | 22 ++ .../server/master/dag/WorkflowDAG.java | 27 ++ .../server/master/dag/WorkflowDAGBuilder.java | 125 ++++++++ .../server/master/dag/WorkflowDAGFactory.java | 65 ++++ .../server/master/dag/WorkflowEngine.java | 82 +++++ .../WorkflowExecuteRunnableRepository.java | 52 ++++ .../master/dag/WorkflowExecutionContext.java | 47 +++ .../dag/WorkflowExecutionContextFactory.java | 31 ++ .../master/dag/WorkflowExecutionDAG.java | 116 +++++++ .../dag/WorkflowExecutionDAGFactory.java | 60 ++++ .../master/dag/WorkflowExecutionRunnable.java | 64 ++++ .../dag/WorkflowExecutionRunnableFactory.java | 40 +++ .../server/master/dag/WorkflowIdentify.java | 35 +++ .../server/master/events/EventEngine.java | 111 +++++++ .../server/master/events/EventFirer.java | 95 ++++++ .../master/events/EventOperatorManager.java | 36 +++ .../server/master/events/IAsyncEvent.java | 24 ++ .../server/master/events/IDelayEvent.java | 26 ++ .../server/master/events/IEvent.java | 22 ++ .../server/master/events/IEventFirer.java | 35 +++ .../server/master/events/IEventOperator.java | 32 ++ .../master/events/IEventOperatorManager.java | 33 ++ .../master/events/IEventRepository.java | 37 +++ .../server/master/events/ISyncEvent.java | 21 ++ .../server/master/events/ITaskEvent.java | 26 ++ .../master/events/ITaskEventOperator.java | 22 ++ .../server/master/events/IWorkflowEvent.java | 29 ++ .../master/events/IWorkflowEventOperator.java | 23 ++ .../master/events/MemoryEventRepository.java | 60 ++++ .../events/TaskLogSendToRemoteEvent.java | 36 +++ .../TaskLogSendToRemoteEventOperator.java | 38 +++ .../master/events/TaskOperationEvent.java | 46 +++ .../events/TaskOperationEventOperator.java | 47 +++ .../master/events/TaskOperationType.java | 29 ++ .../master/events/TaskSuccessEvent.java | 33 ++ .../events/TaskSuccessEventOperator.java | 47 +++ .../master/events/WorkflowFailedEvent.java | 35 +++ .../events/WorkflowFailedEventOperator.java | 57 ++++ .../master/events/WorkflowFinalizeEvent.java | 32 ++ .../events/WorkflowFinalizeEventOperator.java | 42 +++ .../master/events/WorkflowFinishEvent.java | 37 +++ .../master/events/WorkflowOperationEvent.java | 51 ++++ .../WorkflowOperationEventOperator.java | 87 ++++++ .../master/events/WorkflowOperationType.java | 36 +++ .../master/events/WorkflowTimeoutEvent.java | 33 ++ .../events/WorkflowTimeoutEventOperator.java | 61 ++++ .../events/WorkflowTriggerNextTaskEvent.java | 38 +++ .../WorkflowTriggerNextTaskEventOperator.java | 44 +++ .../master/events/WorkflowTriggeredEvent.java | 33 ++ .../WorkflowTriggeredEventOperator.java | 63 ++++ .../TaskExecuteRunnableNotFoundException.java | 29 ++ ...kflowExecuteRunnableNotFoundException.java | 30 ++ .../runner/DefaultTaskExecuteRunnable.java | 3 +- .../runner/TaskExecutionContextFactory.java | 3 +- .../DefaultTaskExecuteRunnableFactory.java | 18 +- .../server/master/utils/ExceptionUtils.java | 31 ++ .../server/master/dag/DAGNodeTest.java | 121 ++++++++ .../MockWorkflowExecutionContextFactory.java | 88 ++++++ .../MockWorkflowExecutionRunnableFactory.java | 31 ++ .../master/dag/WorkflowDAGAssertion.java | 171 +++++++++++ .../master/dag/WorkflowDAGBuilderTest.java | 282 ++++++++++++++++++ .../server/master/dag/WorkflowEngineIT.java | 22 ++ .../server/master/dag/WorkflowEngineTest.java | 114 +++++++ .../WorkflowExecutionRunnableAssertions.java | 48 +++ ...ExecutionRunnableRepositoryAssertions.java | 43 +++ .../dag/WorkflowExecutionRunnableTest.java | 51 ++++ .../events/WorkflowOperationEventTest.java | 46 +++ .../master/utils/ExceptionUtilsTest.java | 18 ++ dolphinscheduler-workflow-engine/README.md | 2 + dolphinscheduler-workflow-engine/pom.xml | 36 +++ .../workflow/engine/dag/DAG.java | 146 +++++++++ .../workflow/engine/dag/DAGEdge.java | 53 ++++ .../workflow/engine/dag/DAGNode.java | 92 ++++++ .../engine/dag/DAGNodeDefinition.java | 35 +++ .../workflow/engine/dag/IDAGNode.java | 32 ++ .../workflow/engine/dag/IDAGNodeAction.java | 35 +++ .../workflow/engine/dag/WorkflowDAG.java | 75 +++++ .../engine/dag/WorkflowDAGBuilder.java | 107 +++++++ .../workflow/engine/engine/DAGEngine.java | 171 +++++++++++ .../engine/engine/DAGEngineFactory.java | 38 +++ .../workflow/engine/engine/IDAGEngine.java | 72 +++++ .../engine/engine/IDAGEngineFactory.java | 26 ++ .../engine/engine/IWorkflowEngine.java | 58 ++++ .../engine/engine/IWorkflowEngineFactory.java | 24 ++ .../engine/engine/WorkflowEngine.java | 82 +++++ .../engine/engine/WorkflowEngineFactory.java | 28 ++ .../workflow/engine/event/EventEngine.java | 111 +++++++ .../workflow/engine/event/EventFirer.java | 91 ++++++ .../engine/event/EventOperatorManager.java | 36 +++ .../workflow/engine/event/IAsyncEvent.java | 24 ++ .../workflow/engine/event/IDelayEvent.java | 26 ++ .../workflow/engine/event/IEvent.java | 22 ++ .../workflow/engine/event/IEventFirer.java | 35 +++ .../workflow/engine/event/IEventOperator.java | 32 ++ .../engine/event/IEventOperatorManager.java | 33 ++ .../engine/event/IEventRepository.java | 37 +++ .../workflow/engine/event/ISyncEvent.java | 21 ++ .../workflow/engine/event/ITaskEvent.java | 26 ++ .../engine/event/ITaskEventOperator.java | 22 ++ .../workflow/engine/event/IWorkflowEvent.java | 29 ++ .../engine/event/IWorkflowEventOperator.java | 23 ++ .../engine/event/MemoryEventRepository.java | 64 ++++ .../engine/event/TaskOperationEvent.java | 37 +++ .../event/TaskOperationEventOperator.java | 44 +++ .../engine/event/TaskOperationType.java | 29 ++ .../engine/event/WorkflowFailedEvent.java | 35 +++ .../engine/event/WorkflowFinalizeEvent.java | 32 ++ .../event/WorkflowFinalizeEventOperator.java | 41 +++ .../engine/event/WorkflowFinishEvent.java | 37 +++ .../engine/event/WorkflowOperationEvent.java | 51 ++++ .../event/WorkflowOperationEventOperator.java | 84 ++++++ .../engine/event/WorkflowOperationType.java | 36 +++ ...kflowExecuteRunnableNotFoundException.java | 30 ++ .../workflow/engine/utils/ExceptionUtils.java | 28 ++ .../workflow/IEventfulExecutionRunnable.java | 34 +++ .../workflow/ITaskExecutionContext.java | 24 ++ .../ITaskExecutionContextFactory.java | 25 ++ .../workflow/ITaskExecutionRunnable.java | 59 ++++ .../ITaskExecutionRunnableFactory.java | 30 ++ .../ITaskExecutionRunnableRepository.java | 34 +++ .../engine/workflow/ITaskInstance.java | 26 ++ .../IWorkflowExecuteRunnableRepository.java | 33 ++ .../workflow/IWorkflowExecutionContext.java | 38 +++ .../workflow/IWorkflowExecutionDAG.java | 71 +++++ .../IWorkflowExecutionDAGFactory.java | 24 ++ .../workflow/IWorkflowExecutionRunnable.java | 56 ++++ .../IWorkflowExecutionRunnableFactory.java | 24 ++ .../engine/workflow/IWorkflowInstance.java | 26 ++ ...etonWorkflowExecuteRunnableRepository.java | 61 ++++ .../engine/workflow/TaskExecutionContext.java | 32 ++ .../workflow/TaskExecutionContextFactory.java | 27 ++ .../workflow/TaskExecutionRunnable.java | 57 ++++ .../TaskExecutionRunnableFactory.java | 35 +++ .../workflow/WorkflowExecutionContext.java | 39 +++ .../engine/workflow/WorkflowExecutionDAG.java | 124 ++++++++ .../workflow/WorkflowExecutionRunnable.java | 64 ++++ .../WorkflowExecutionRunnableFactory.java | 36 +++ .../engine/workflow/WorkflowInstance.java | 38 +++ ...owExecuteRunnableRepositoryAssertions.java | 37 +++ .../assertions/WorkflowDAGAssertion.java | 173 +++++++++++ .../WorkflowExecutionRunnableAssertions.java | 49 +++ .../workflow/engine/dag/DAGNodeTest.java | 104 +++++++ .../engine/dag/WorkflowDAGBuilderTest.java | 260 ++++++++++++++++ .../engine/engine/WorkflowEngineIT.java | 24 ++ .../engine/engine/WorkflowEngineTest.java | 105 +++++++ .../event/WorkflowOperationEventTest.java | 30 ++ .../MockWorkflowExecutionRunnableFactory.java | 46 +++ pom.xml | 1 + 181 files changed, 9008 insertions(+), 25 deletions(-) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/BasicDAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEdge.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngine.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngineFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGNode.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngine.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngineFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGNodeAction.java rename dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/{exception/TaskExecutionContextCreateException.java => dag/IEventRepositoryFactory.java} (77%) create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventfulExecutionRunnable.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContext.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContextFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnable.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepository.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepositoryFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAGFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowEngine.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionContext.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAGFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionRunnable.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/MemoryEventRepositoryFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionContext.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnable.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepository.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepositoryFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionValve.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskIdentify.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilder.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngine.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecuteRunnableRepository.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContext.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContextFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAG.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAGFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnable.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableFactory.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowIdentify.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IDelayEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/MemoryEventRepository.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationType.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinishEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationType.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEvent.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEventOperator.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java create mode 100644 dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/DAGNodeTest.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionContextFactory.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionRunnableFactory.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGAssertion.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilderTest.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineIT.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineTest.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableAssertions.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableRepositoryAssertions.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableTest.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventTest.java create mode 100644 dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtilsTest.java create mode 100644 dolphinscheduler-workflow-engine/README.md create mode 100644 dolphinscheduler-workflow-engine/pom.xml create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAG.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGEdge.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNode.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeDefinition.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNode.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNodeAction.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAG.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilder.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngine.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngineFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngine.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngineFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngine.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngineFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngine.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventEngine.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventFirer.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventOperatorManager.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IAsyncEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IDelayEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventFirer.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperatorManager.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventRepository.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ISyncEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/MemoryEventRepository.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationType.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFailedEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinishEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEvent.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventOperator.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationType.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/exception/WorkflowExecuteRunnableNotFoundException.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/utils/ExceptionUtils.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IEventfulExecutionRunnable.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContext.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContextFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnable.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableRepository.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskInstance.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecuteRunnableRepository.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionContext.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAG.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAGFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnable.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnableFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowInstance.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/SingletonWorkflowExecuteRunnableRepository.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContext.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContextFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnable.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnableFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionContext.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionDAG.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnable.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnableFactory.java create mode 100644 dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowInstance.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/SingletonWorkflowExecuteRunnableRepositoryAssertions.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowDAGAssertion.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowExecutionRunnableAssertions.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeTest.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilderTest.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineIT.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineTest.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventTest.java create mode 100644 dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/workflow/MockWorkflowExecutionRunnableFactory.java diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java index d8f01f473f..8075dc4091 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java @@ -34,6 +34,7 @@ import java.util.Date; import java.util.List; import lombok.AllArgsConstructor; +import lombok.Builder; import lombok.Data; import lombok.NoArgsConstructor; @@ -44,11 +45,10 @@ import com.baomidou.mybatisplus.annotation.TableId; import com.baomidou.mybatisplus.annotation.TableName; import com.google.common.base.Strings; -/** - * process instance - */ -@NoArgsConstructor @Data +@Builder +@AllArgsConstructor +@NoArgsConstructor @TableName("t_ds_process_instance") public class ProcessInstance { diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java index 8bd12026f0..40955a8a67 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java @@ -27,6 +27,9 @@ public interface ProcessTaskRelationLogDao extends IDao List queryByWorkflowDefinitionCode(long workflowDefinitionCode); + List queryByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode, + int workflowDefinitionVerison); + void deleteByWorkflowDefinitionCode(long workflowDefinitionCode); /** diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java index 81c29b89da..b800acb5c7 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java @@ -44,6 +44,12 @@ public class ProcessTaskRelationLogDaoImpl extends BaseDao queryByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode, + int workflowDefinitionVerison) { + return mybatisMapper.queryByProcessCodeAndVersion(workflowDefinitionCode, workflowDefinitionVerison); + } + @Override public void deleteByWorkflowDefinitionCode(long workflowDefinitionCode) { mybatisMapper.deleteByWorkflowDefinitionCode(workflowDefinitionCode); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/BasicDAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/BasicDAG.java new file mode 100644 index 0000000000..26037eec17 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/BasicDAG.java @@ -0,0 +1,71 @@ +/* + * 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.dag; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import lombok.experimental.SuperBuilder; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@SuperBuilder +public abstract class BasicDAG implements DAG { + + protected Map dagNodeMap; + + @Override + public List getDirectPostNodes(DAGNode dagNode) { + final String nodeName = dagNode.getNodeName(); + if (!dagNodeMap.containsKey(nodeName)) { + return Collections.emptyList(); + } + DAGNode node = dagNodeMap.get(nodeName); + List dagNodes = new ArrayList<>(); + for (DAGEdge edge : node.getOutDegrees()) { + if (dagNodeMap.containsKey(edge.getToNodeName())) { + dagNodes.add(dagNodeMap.get(edge.getToNodeName())); + } + } + return dagNodes; + } + + @Override + public List getDirectPreNodes(DAGNode dagNode) { + final String nodeName = dagNode.getNodeName(); + if (!dagNodeMap.containsKey(nodeName)) { + return Collections.emptyList(); + } + DAGNode node = dagNodeMap.get(nodeName); + List dagNodes = new ArrayList<>(); + for (DAGEdge edge : node.getInDegrees()) { + if (dagNodeMap.containsKey(edge.getFromNodeName())) { + dagNodes.add(dagNodeMap.get(edge.getFromNodeName())); + } + } + return dagNodes; + } + + @Override + public DAGNode getDAGNode(String nodeName) { + return dagNodeMap.get(nodeName); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAG.java new file mode 100644 index 0000000000..40dee4f119 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAG.java @@ -0,0 +1,146 @@ +/* + * 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.dag; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * The Directed Acyclic Graph class. + *

+ * The DAG is a directed graph, which contains the nodes and the edges, the nodeName is the unique identifier of the node. + * The nodes are the tasks, the edges are the dependencies between the tasks. + * The DAG is acyclic, which means there is no cycle in the graph. + * The DAG is a directed graph, which means the edges have direction. + * + */ +public interface DAG { + + /** + * Get the direct post node of given dagNode, if the dagNode is null, return the nodes which doesn't have inDegrees. + * e.g. The DAG is: + *

+     *      {@code
+     *          1 -> 2 -> 3
+     *          4 -> 5
+     *          6
+     *      }
+     * 
+ *
  • The post node of 1 is 2. + *
  • The post node of 3 is empty. + *
  • The post node of null is 1,4,6. + * + * @param dagNode the node of the DAG, can be null. + * @return post node list, sort by priority. + */ + List getDirectPostNodes(DAGNode dagNode); + + /** + * Same with {@link #getDirectPostNodes(DAGNode)}. + *

    + * If the dagNodeName is null, return the nodes which doesn't have inDegrees. Otherwise, return the post nodes of + * the given dagNodeName. If the dagNodeName is not null and cannot find the node in DAG, throw IllegalArgumentException. + * + * @param dagNodeName task name, can be null. + * @return post task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPostNodes(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPostNodes(dagNode); + } + + /** + * Same with {@link #getDirectPostNodes(String)}. Return the post node names. + * + * @param dagNodeName task name, can be null. + * @return post task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPostNodeNames(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPostNodes(dagNode).stream().map(DAGNode::getNodeName).collect(Collectors.toList()); + } + + /** + * Get the direct pre node of given dagNode, if the dagNode is null, return the nodes which doesn't have outDegrees. + * e.g. The DAG is: + *

    +     *      {@code
    +     *          1 -> 2 -> 3
    +     *          4 -> 5
    +     *          6
    +     *      }
    +     * 
    + *
  • The pre node of 1 is empty. + *
  • The pre node of 3 is 2. + *
  • The pre node of null is 3,5,6. + * + * @param dagNode the node of the DAG, can be null. + * @return pre node list, sort by priority. + */ + List getDirectPreNodes(DAGNode dagNode); + + /** + * Same with {@link #getDirectPreNodes(DAGNode)}. + *

    + * If the dagNodeName is null, return the nodes which doesn't have outDegrees. Otherwise, return the pre nodes of + * the given dagNodeName. If the dagNodeName is not null and cannot find the node in DAG, throw IllegalArgumentException. + * + * @param dagNodeName task name, can be null. + * @return pre task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPreNodes(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPreNodes(dagNode); + } + + /** + * Same with {@link #getDirectPreNodes(String)}. Return the pre node names. + * + * @param dagNodeName task name, can be null. + * @return pre task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPreNodeNames(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPreNodes(dagNode).stream().map(DAGNode::getNodeName).collect(Collectors.toList()); + } + + /** + * Get the node of the DAG by the node name. + * + * @param nodeName the node name. + * @return the node of the DAG, return null if cannot find the node. + */ + DAGNode getDAGNode(String nodeName); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEdge.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEdge.java new file mode 100644 index 0000000000..d3ede91fef --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEdge.java @@ -0,0 +1,51 @@ +/* + * 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.dag; + +import lombok.Builder; +import lombok.Data; + +/** + * The edge of the DAG. + *

    + * The edge contains the fromNodeName and the toNodeName, the fromNodeName is the node name of the from node, the toNodeName is the node name of the to node. + *

    + * The formNodeName can be null, which means the edge is from the start node of the DAG. + * The toNodeName can be null, which means the edge is to the end node of the DAG. + * The fromNodeName and the toNodeName cannot be null at the same time. + */ +@Data +@Builder +public class DAGEdge { + + private String fromNodeName; + private String toNodeName; + + public DAGEdge(String fromNodeName, String toNodeName) { + if (fromNodeName == null && toNodeName == null) { + throw new IllegalArgumentException("fromNodeName and toNodeName cannot be null at the same time" + + "fromNodeName: " + fromNodeName + ", toNodeName: " + toNodeName); + } + if (fromNodeName != null && fromNodeName.equals(toNodeName)) { + throw new IllegalArgumentException("fromNodeName and toNodeName cannot be the same" + + "fromNodeName: " + fromNodeName + ", toNodeName: " + toNodeName); + } + this.fromNodeName = fromNodeName; + this.toNodeName = toNodeName; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngine.java new file mode 100644 index 0000000000..cf1daca1f7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngine.java @@ -0,0 +1,148 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.server.master.events.TaskOperationEvent; +import org.apache.dolphinscheduler.server.master.events.TaskOperationType; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DAGEngine implements IDAGEngine { + + private final IWorkflowExecutionContext workflowExecutionContext; + + public DAGEngine(IWorkflowExecutionContext workflowExecutionContext) { + this.workflowExecutionContext = workflowExecutionContext; + } + + @Override + public void triggerNextTasks(String parentTaskNodeName) { + workflowExecutionContext.getWorkflowExecutionDAG() + .getDirectPostNodeNames(parentTaskNodeName) + .forEach(this::triggerTask); + } + + @Override + public void triggerTask(String taskName) { + IWorkflowExecutionDAG workflowExecutionDAG = workflowExecutionContext.getWorkflowExecutionDAG(); + DAGNode dagNode = workflowExecutionDAG.getDAGNode(taskName); + if (dagNode == null) { + log.error("Cannot find the DAGNode for task: {}", taskName); + return; + } + // Use condition check? + if (!workflowExecutionDAG.isTaskAbleToBeTriggered(taskName)) { + log.info("The task: {} is not able to be triggered", taskName); + return; + } + + if (dagNode.isSkip()) { + log.info("The task: {} is skipped", taskName); + triggerNextTasks(taskName); + return; + } + + TaskExecutionRunnable taskExecutionRunnable = workflowExecutionDAG.triggerTask(taskName); + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.RUN) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void failoverTask(Integer taskInstanceId) { + TaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.FAILOVER) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void retryTask(Integer taskInstanceId) { + TaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.RETRY) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void pauseAllTask() { + workflowExecutionContext.getWorkflowExecutionDAG() + .getActiveTaskExecutionRunnable() + .stream() + .map(taskExecutionRunnable -> taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId()) + .forEach(this::pauseTask); + } + + @Override + public void pauseTask(Integer taskInstanceId) { + TaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.PAUSE) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void killAllTask() { + workflowExecutionContext.getWorkflowExecutionDAG() + .getActiveTaskExecutionRunnable() + .stream() + .map(taskExecutionRunnable -> taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId()) + .forEach(this::killTask); + } + + @Override + public void killTask(Integer taskInstanceId) { + TaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.KILL) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngineFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngineFactory.java new file mode 100644 index 0000000000..43b1a3e5c0 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngineFactory.java @@ -0,0 +1,32 @@ +/* + * 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.dag; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class DAGEngineFactory implements IDAGEngineFactory { + + @Override + public IDAGEngine createDAGEngine(IWorkflowExecutionContext workflowExecutionContext) { + return new DAGEngine(workflowExecutionContext); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGNode.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGNode.java new file mode 100644 index 0000000000..95bd6c75ba --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGNode.java @@ -0,0 +1,92 @@ +/* + * 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.dag; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; + +import lombok.Builder; +import lombok.Getter; +import lombok.NoArgsConstructor; + +/** + * The node of the DAG. + *

    + * The node contains the node name, the content of the node, the inDegrees and the outDegrees. + * The inDegrees is the edge from other nodes to the current node, the outDegrees is the edge from the current + * node to other nodes. + */ +@Getter +@Builder +@NoArgsConstructor +public class DAGNode { + + private String nodeName; + + /** + * whether the node is skipped, default is false, which means the node is not skipped. + * If the node is skipped, the node will not be executed. + */ + @Builder.Default + private boolean skip = false; + + private List inDegrees; + private List outDegrees; + + public DAGNode(String nodeName, + List inDegrees, + List outDegrees) { + this(nodeName, false, inDegrees, outDegrees); + } + + public DAGNode(String nodeName, + boolean skip, + List inDegrees, + List outDegrees) { + if (StringUtils.isEmpty(nodeName)) { + throw new IllegalArgumentException("nodeName cannot be empty"); + } + + if (CollectionUtils.isNotEmpty(inDegrees)) { + inDegrees.forEach(dagEdge -> { + if (!nodeName.equals(dagEdge.getToNodeName())) { + throw new IllegalArgumentException( + "The toNodeName of inDegree should be the nodeName of the node: " + + nodeName + ", inDegree: " + dagEdge); + } + }); + } + + if (CollectionUtils.isNotEmpty(outDegrees)) { + outDegrees.forEach(dagEdge -> { + if (!nodeName.equals(dagEdge.getFromNodeName())) { + throw new IllegalArgumentException( + "The fromNodeName of outDegree should be the nodeName of the node: " + + nodeName + ", outDegree: " + dagEdge); + } + }); + } + + this.nodeName = nodeName; + this.inDegrees = inDegrees; + this.outDegrees = outDegrees; + this.skip = skip; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngine.java new file mode 100644 index 0000000000..7adc2d9a1e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngine.java @@ -0,0 +1,70 @@ +/* + * 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.dag; + +/** + * The IDAGEngine is responsible for triggering, killing, pausing, and finalizing task in {@link IWorkflowExecutionDAG}. + *

    All DAG operation should directly use the method in IDAGEngine, new {@link IWorkflowExecutionDAG} should be triggered by new IDAGEngine. + */ +public interface IDAGEngine { + + /** + * Trigger the tasks which are post of the given task. + *

    If there are no task after the given taskNode, will try to finish the WorkflowExecutionRunnable. + *

    If the + * + * @param parentTaskNodeName the parent task name + */ + void triggerNextTasks(String parentTaskNodeName); + + /** + * Trigger the given task + * + * @param taskName task name + */ + void triggerTask(String taskName); + + /** + * Failover the given task. + * + * @param taskInstanceId taskInstanceId + */ + void failoverTask(Integer taskInstanceId); + + /** + * Retry the given task. + * + * @param taskInstanceId taskInstanceId + */ + void retryTask(Integer taskInstanceId); + + void pauseAllTask(); + + /** + * Pause the given task. + */ + void pauseTask(Integer taskInstanceId); + + void killAllTask(); + + /** + * Kill the given task. + */ + void killTask(Integer taskId); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngineFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngineFactory.java new file mode 100644 index 0000000000..0ae555ea8d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngineFactory.java @@ -0,0 +1,24 @@ +/* + * 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.dag; + +public interface IDAGEngineFactory { + + IDAGEngine createDAGEngine(IWorkflowExecutionContext workflowExecutionContext); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGNodeAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGNodeAction.java new file mode 100644 index 0000000000..70148bd737 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGNodeAction.java @@ -0,0 +1,35 @@ +/* + * 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.dag; + +/** + * The IDAGNodeAction represent the action of a DAG node. + */ +public interface IDAGNodeAction { + + void run(); + + void kill(); + + void pause(); + + void success(); + + void failure(); + +} 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/dag/IEventRepositoryFactory.java similarity index 77% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventRepositoryFactory.java index ac37c94438..630d412856 100644 --- 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/dag/IEventRepositoryFactory.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.exception; +package org.apache.dolphinscheduler.server.master.dag; -public class TaskExecutionContextCreateException extends MasterException { +import org.apache.dolphinscheduler.server.master.events.IEventRepository; - public TaskExecutionContextCreateException(String message) { - super(message); - } +public interface IEventRepositoryFactory { + + IEventRepository createEventRepository(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventfulExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventfulExecutionRunnable.java new file mode 100644 index 0000000000..952fa23120 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventfulExecutionRunnable.java @@ -0,0 +1,34 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.server.master.events.IEvent; +import org.apache.dolphinscheduler.server.master.events.IEventRepository; + +public interface IEventfulExecutionRunnable { + + IEventRepository getEventRepository(); + + default void storeEventToTail(IEvent event) { + getEventRepository().storeEventToTail(event); + } + + default void storeEventToHead(IEvent event) { + getEventRepository().storeEventToHead(event); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContext.java new file mode 100644 index 0000000000..818cbd0e10 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContext.java @@ -0,0 +1,26 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; + +public interface ITaskExecutionContext { + + TaskInstance getTaskInstance(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContextFactory.java new file mode 100644 index 0000000000..7c67307371 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContextFactory.java @@ -0,0 +1,24 @@ +/* + * 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.dag; + +public interface ITaskExecutionContextFactory { + + ITaskExecutionContext createTaskExecutionContext(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnable.java new file mode 100644 index 0000000000..dceff535c5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnable.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +/** + * The TaskExecutionRunnable represent the running task, it is responsible for operate the task instance. e.g. dispatch, kill, pause. + */ +public interface ITaskExecutionRunnable { + + /** + * Dispatch the task instance. + */ + void dispatch(); + + /** + * Run the task instance. + */ + void run(); + + /** + * Kill the task instance. + */ + void kill(); + + /** + * Pause the task instance. + */ + void pause(); + + /** + * Get the task execution context. + * + * @return the task execution context + */ + TaskExecutionContext getTaskExecutionContext(); + + /** + * Determine whether the current task can be accessed to the post task. + * + * @param taskNodeName post task name + * @return true if the current task can be accessed to the post task. + */ + boolean canAccessTo(String taskNodeName); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableFactory.java new file mode 100644 index 0000000000..d1d2cf1ae9 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableFactory.java @@ -0,0 +1,24 @@ +/* + * 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.dag; + +public interface ITaskExecutionRunnableFactory { + + TaskExecutionRunnable createTaskExecutionRunnable(ITaskExecutionContext taskExecutionContext); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepository.java new file mode 100644 index 0000000000..094b752cf1 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepository.java @@ -0,0 +1,33 @@ +/* + * 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.dag; + +import java.util.Collection; + +public interface ITaskExecutionRunnableRepository { + + void storeTaskExecutionRunnable(TaskExecutionRunnable taskExecutionRunnable); + + TaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId); + + TaskExecutionRunnable getTaskExecutionRunnableByName(String taskInstanceName); + + Collection getActiveTaskExecutionRunnable(); + + void removeTaskExecutionRunnable(Integer taskInstanceId); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepositoryFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepositoryFactory.java new file mode 100644 index 0000000000..6e237a2b06 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepositoryFactory.java @@ -0,0 +1,24 @@ +/* + * 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.dag; + +public interface ITaskExecutionRunnableRepositoryFactory { + + TaskExecutionRunnableRepository createTaskExecutionRunnableRepository(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAG.java new file mode 100644 index 0000000000..fca3010035 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAG.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +/** + * The IWorkflowDAG represent the DAG of a workflow. + */ +public interface IWorkflowDAG extends DAG { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAGFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAGFactory.java new file mode 100644 index 0000000000..00c9e237da --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAGFactory.java @@ -0,0 +1,35 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; + +/** + * The Factory used to create {@link IWorkflowDAG} + */ +public interface IWorkflowDAGFactory { + + /** + * Create the WorkflowDAG + * + * @param workflowInstance workflowInstance. + * @return workflow DAG. + */ + IWorkflowDAG createWorkflowDAG(ProcessInstance workflowInstance); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowEngine.java new file mode 100644 index 0000000000..0c3f38dce9 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowEngine.java @@ -0,0 +1,57 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; + +/** + * The WorkflowEngine is responsible for starting, stopping, pausing, and finalizing workflows. + */ +public interface IWorkflowEngine { + + /** + * 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. Once a workflow has been finalized, then it cannot receive new operation, and will be removed from memory. + * + * @param workflowInstanceId the ID of the workflow to finalize + */ + void finalizeWorkflow(Integer workflowInstanceId); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionContext.java new file mode 100644 index 0000000000..53bcf3555b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionContext.java @@ -0,0 +1,46 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.server.master.events.IEventRepository; + +import java.util.List; + +public interface IWorkflowExecutionContext { + + IWorkflowExecutionDAG getWorkflowExecutionDAG(); + + ProcessInstance getWorkflowInstance(); + + ProcessDefinition getWorkflowDefinition(); + + List getBeginNodeNames(); + + IEventRepository getEventRepository(); + + default int getWorkflowInstanceId() { + return getWorkflowInstance().getId(); + } + + default String getWorkflowInstanceName() { + return getWorkflowInstance().getName(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAG.java new file mode 100644 index 0000000000..33e597cf2e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAG.java @@ -0,0 +1,74 @@ +/* + * 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.dag; + +import java.util.List; + +/** + * The WorkflowExecutionDAG represent the running workflow DAG. + */ +public interface IWorkflowExecutionDAG extends DAG { + + /** + * Trigger the taskNode by given taskName. + * + * @param taskName taskNodeName + * @return TaskExecutionRunnable + */ + TaskExecutionRunnable triggerTask(String taskName); + + /** + * Get TaskExecutionRunnable by given TaskInstanceId. + * + * @param taskInstanceId taskInstanceId. + * @return TaskExecutionRunnable + */ + TaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId); + + /** + * Get TaskExecutionRunnable by given taskName. + * + * @param taskName task name. + * @return TaskExecutionRunnable + */ + TaskExecutionRunnable getTaskExecutionRunnableByName(String taskName); + + /** + * Get TaskExecutionRunnable which is not finished. + * + * @return TaskExecutionRunnable + */ + List getActiveTaskExecutionRunnable(); + + /** + * Get the direct pre TaskExecutionRunnable of the given taskName. + * + * @param taskName task name. + * @return TaskExecutionRunnable + */ + List getDirectPreTaskExecutionRunnable(String taskName); + + /** + * Check whether the taskNode is ready to run. + * + * @param taskName taskNodeName + * @return true if the taskNode is ready to run. + */ + boolean isTaskAbleToBeTriggered(String taskName); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAGFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAGFactory.java new file mode 100644 index 0000000000..429987ca1f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAGFactory.java @@ -0,0 +1,23 @@ +/* + * 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.dag; + +public interface IWorkflowExecutionDAGFactory { + + IWorkflowExecutionDAG createWorkflowExecutionDAG(); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionRunnable.java new file mode 100644 index 0000000000..d0a641239d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionRunnable.java @@ -0,0 +1,54 @@ +/* + * 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.dag; + +/** + * The IWorkflowExecuteRunnable represent a running workflow instance, it is responsible for operate the workflow instance. e.g. start, kill, pause. + */ +public interface IWorkflowExecutionRunnable extends IEventfulExecutionRunnable { + + /** + * Start the workflow instance. + */ + void start(); + + /** + * Kill the workflow instance. + */ + void kill(); + + /** + * Pause the workflow instance. + */ + void pause(); + + /** + * Get the workflow execution context. + * + * @return the workflow execution context + */ + IWorkflowExecutionContext getWorkflowExecutionContext(); + + /** + * Get the {@link IDAGEngine} which used to execute the dag of the workflow instance. + * + * @return dag engine. + */ + IDAGEngine getDagEngine(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/MemoryEventRepositoryFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/MemoryEventRepositoryFactory.java new file mode 100644 index 0000000000..e61351f472 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/MemoryEventRepositoryFactory.java @@ -0,0 +1,35 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.server.master.events.IEventRepository; +import org.apache.dolphinscheduler.server.master.events.MemoryEventRepository; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class MemoryEventRepositoryFactory implements IEventRepositoryFactory { + + @Override + public IEventRepository createEventRepository() { + return new MemoryEventRepository(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionContext.java new file mode 100644 index 0000000000..bf079e861f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionContext.java @@ -0,0 +1,35 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@AllArgsConstructor +@NoArgsConstructor +public class TaskExecutionContext implements ITaskExecutionContext { + + private TaskInstance taskInstance; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnable.java new file mode 100644 index 0000000000..0f21592b26 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnable.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +import lombok.Builder; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Getter +@Builder +public class TaskExecutionRunnable implements ITaskExecutionRunnable { + + private TaskExecutionContext taskExecutionContext; + + @Override + public void dispatch() { + // todo: check if the operation is valid + } + + @Override + public void run() { + + } + + @Override + public void kill() { + // todo: check if the operation is valid + } + + @Override + public void pause() { + // todo: check if the operation is valid + } + + @Override + public boolean canAccessTo(String postTaskNodeName) { + return false; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepository.java new file mode 100644 index 0000000000..eb8799c212 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepository.java @@ -0,0 +1,68 @@ +/* + * 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.dag; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * Use to store the TaskExecutionRunnable of a DAG. + */ +public class TaskExecutionRunnableRepository implements ITaskExecutionRunnableRepository { + + private final Map taskExecuteRunnableMap = new ConcurrentHashMap<>(); + + private final Map taskExecuteRunnableNameMap = new ConcurrentHashMap<>(); + + @Override + public void storeTaskExecutionRunnable(TaskExecutionRunnable taskExecutionRunnable) { + taskExecuteRunnableMap.put(taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId(), + taskExecutionRunnable); + taskExecuteRunnableNameMap.put(taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getName(), + taskExecutionRunnable); + } + + public TaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId) { + return taskExecuteRunnableMap.get(taskInstanceId); + } + + @Override + public TaskExecutionRunnable getTaskExecutionRunnableByName(String taskInstanceName) { + return taskExecuteRunnableNameMap.get(taskInstanceName); + } + + public Collection getActiveTaskExecutionRunnable() { + return taskExecuteRunnableMap.values() + .stream() + .filter(taskExecutionRunnable -> { + return taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getState().isRunning(); + }) + .collect(Collectors.toList()); + } + + public void removeTaskExecutionRunnable(Integer taskInstanceId) { + TaskExecutionRunnable taskExecutionRunnable = taskExecuteRunnableMap.remove(taskInstanceId); + if (taskExecutionRunnable != null) { + taskExecuteRunnableNameMap + .remove(taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getName()); + } + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepositoryFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepositoryFactory.java new file mode 100644 index 0000000000..55ff26916d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepositoryFactory.java @@ -0,0 +1,32 @@ +/* + * 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.dag; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskExecutionRunnableRepositoryFactory implements ITaskExecutionRunnableRepositoryFactory { + + public TaskExecutionRunnableRepository createTaskExecutionRunnableRepository() { + return new TaskExecutionRunnableRepository(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionValve.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionValve.java new file mode 100644 index 0000000000..2bf015992d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionValve.java @@ -0,0 +1,27 @@ +/* + * 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.dag; + +/** + * Used to judge whether the task can be executed. + */ +public interface TaskExecutionValve { + + boolean canOpen(TaskExecutionRunnable preTask, String postTask); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskIdentify.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskIdentify.java new file mode 100644 index 0000000000..d9dd0d0b1c --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskIdentify.java @@ -0,0 +1,22 @@ +/* + * 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.dag; + +public class TaskIdentify { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAG.java new file mode 100644 index 0000000000..c48785b795 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAG.java @@ -0,0 +1,27 @@ +/* + * 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.dag; + +import lombok.experimental.SuperBuilder; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@SuperBuilder +public class WorkflowDAG extends BasicDAG implements IWorkflowDAG { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilder.java new file mode 100644 index 0000000000..1906afc7fb --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilder.java @@ -0,0 +1,125 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.common.enums.Flag; +import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog; +import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Used to build WorkflowDAG, you need to add TaskNode first, then add TaskEdge. + * After adding all the TaskNodes and TaskEdges, you can call the build method to get the WorkflowDAG. + *

    + * Example: + *

    + *     {@code
    + *          WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    + *                 .addTaskNode(taskNodeA)
    + *                 .addTaskNode(taskNodeB)
    + *                 .addTaskNode(taskNodeC)
    + *                 .addTaskEdge(edgeAB)
    + *                 .addTaskEdge(edgeBC)
    + *                 .build();
    + *     }
    + * 
    + */ +public class WorkflowDAGBuilder { + + private final Map taskNameMap; + + private final Map taskCodeMap; + + private WorkflowDAGBuilder() { + this.taskCodeMap = new HashMap<>(); + this.taskNameMap = new HashMap<>(); + } + + public static WorkflowDAGBuilder newBuilder() { + return new WorkflowDAGBuilder(); + } + + public WorkflowDAGBuilder addTaskNodes(List taskDefinitionList) { + taskDefinitionList.forEach(this::addTaskNode); + return this; + } + + public WorkflowDAGBuilder addTaskNode(TaskDefinitionLog taskDefinition) { + String taskName = taskDefinition.getName(); + long taskCode = taskDefinition.getCode(); + if (taskCodeMap.containsKey(taskCode)) { + throw new IllegalArgumentException("TaskNode with code " + taskCode + " already exists"); + } + if (taskNameMap.containsKey(taskName)) { + throw new IllegalArgumentException("TaskNode with name " + taskName + " already exists"); + } + + DAGNode taskNode = DAGNode.builder() + .nodeName(taskName) + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .skip(Flag.NO.equals(taskDefinition.getFlag())) + .build(); + taskNameMap.put(taskName, taskNode); + taskCodeMap.put(taskCode, taskNode); + return this; + } + + public WorkflowDAGBuilder addTaskEdges(List processTaskRelations) { + processTaskRelations.forEach(this::addTaskEdge); + return this; + } + + public WorkflowDAGBuilder addTaskEdge(ProcessTaskRelationLog processTaskRelation) { + long preTaskCode = processTaskRelation.getPreTaskCode(); + long postTaskCode = processTaskRelation.getPostTaskCode(); + + if (taskCodeMap.containsKey(preTaskCode)) { + DAGNode fromTask = taskCodeMap.get(preTaskCode); + if (taskCodeMap.containsKey(postTaskCode)) { + DAGNode toTask = taskCodeMap.get(postTaskCode); + DAGEdge edge = DAGEdge.builder() + .fromNodeName(fromTask.getNodeName()) + .toNodeName(toTask.getNodeName()) + .build(); + if (fromTask.getOutDegrees().contains(edge)) { + throw new IllegalArgumentException( + "Edge from " + fromTask.getNodeName() + " to " + toTask.getNodeName() + " already exists"); + } + fromTask.getOutDegrees().add(edge); + if (toTask.getInDegrees().contains(edge)) { + throw new IllegalArgumentException( + "Edge from " + fromTask.getNodeName() + " to " + toTask.getNodeName() + " already exists"); + } + toTask.getInDegrees().add(edge); + } + } + return this; + } + + public WorkflowDAG build() { + return WorkflowDAG.builder() + .dagNodeMap(taskNameMap) + .build(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGFactory.java new file mode 100644 index 0000000000..de743c2851 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGFactory.java @@ -0,0 +1,65 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog; +import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog; +import org.apache.dolphinscheduler.dao.repository.ProcessTaskRelationLogDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao; + +import java.util.List; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowDAGFactory implements IWorkflowDAGFactory { + + @Autowired + private TaskDefinitionLogDao taskDefinitionLogDao; + + @Autowired + private ProcessTaskRelationLogDao processTaskRelationLogDao; + + @Override + public IWorkflowDAG createWorkflowDAG(ProcessInstance processInstance) { + Long workflowDefinitionCode = processInstance.getProcessDefinitionCode(); + Integer workflowDefinitionVersion = processInstance.getProcessDefinitionVersion(); + List taskDefinitions = queryTaskNodes(workflowDefinitionCode, workflowDefinitionVersion); + List taskRelations = queryTaskEdges(workflowDefinitionCode, workflowDefinitionVersion); + return WorkflowDAGBuilder.newBuilder() + .addTaskNodes(taskDefinitions) + .addTaskEdges(taskRelations) + .build(); + } + + private List queryTaskNodes(Long workflowDefinitionCode, Integer workflowDefinitionVersion) { + return taskDefinitionLogDao.queryByWorkflowDefinitionCodeAndVersion(workflowDefinitionCode, + workflowDefinitionVersion); + } + + private List queryTaskEdges(Long workflowDefinitionCode, + Integer workflowDefinitionVersion) { + return processTaskRelationLogDao.queryByWorkflowDefinitionCodeAndVersion(workflowDefinitionCode, + workflowDefinitionVersion); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngine.java new file mode 100644 index 0000000000..550c837ffc --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngine.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +import org.apache.dolphinscheduler.server.master.events.WorkflowOperationEvent; +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowEngine implements IWorkflowEngine { + + private final WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + public WorkflowEngine(WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) { + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + } + + @Override + public void triggerWorkflow(IWorkflowExecutionRunnable workflowExecuteRunnable) { + IWorkflowExecutionContext workflowExecutionContext = workflowExecuteRunnable.getWorkflowExecutionContext(); + Integer workflowInstanceId = workflowExecutionContext.getWorkflowInstanceId(); + log.info("Triggering WorkflowExecutionRunnable: {}", workflowExecutionContext.getWorkflowInstanceName()); + workflowExecuteRunnableRepository.storeWorkflowExecutionRunnable(workflowExecuteRunnable); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.triggerEvent(workflowInstanceId)); + } + + @Override + public void pauseWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Pausing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.pauseEvent(workflowInstanceId)); + } + + @Override + public void killWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Killing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.killEvent(workflowInstanceId)); + } + + @Override + public void finalizeWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecutionRunnable == null) { + return; + } + log.info("Finalizing WorkflowExecutionRunnable: {}", + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnableRepository.removeWorkflowExecutionRunnable(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecuteRunnableRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecuteRunnableRepository.java new file mode 100644 index 0000000000..78beaa44e7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecuteRunnableRepository.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecuteRunnableRepository { + + private final Map workflowExecutionRunnableMap = new ConcurrentHashMap<>(); + + public void storeWorkflowExecutionRunnable(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnableMap.put( + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstanceId(), + workflowExecutionRunnable); + } + + public IWorkflowExecutionRunnable getWorkflowExecutionRunnableById(Integer workflowInstanceId) { + return workflowExecutionRunnableMap.get(workflowInstanceId); + } + + public Collection getActiveWorkflowExecutionRunnable() { + return workflowExecutionRunnableMap.values(); + } + + public void removeWorkflowExecutionRunnable(Integer workflowInstanceId) { + workflowExecutionRunnableMap.remove(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContext.java new file mode 100644 index 0000000000..be9c165882 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContext.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.dag; + +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.server.master.events.IEventRepository; + +import java.util.List; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowExecutionContext implements IWorkflowExecutionContext { + + private ProcessDefinition workflowDefinition; + + private ProcessInstance workflowInstance; + + private List beginNodeNames; + + private IWorkflowExecutionDAG workflowExecutionDAG; + + private IEventRepository eventRepository; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContextFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContextFactory.java new file mode 100644 index 0000000000..69ad93b106 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContextFactory.java @@ -0,0 +1,31 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.dao.entity.Command; + +import org.springframework.stereotype.Component; + +@Component +public class WorkflowExecutionContextFactory { + + public WorkflowExecutionContext createWorkflowExecutionContext(Command command) { + return null; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAG.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAG.java new file mode 100644 index 0000000000..86c690d44f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAG.java @@ -0,0 +1,116 @@ +/* + * 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.dag; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import lombok.experimental.SuperBuilder; +import lombok.extern.slf4j.Slf4j; + +/** + * The WorkflowExecutionDAG represent a running workflow instance DAG. + */ +@Slf4j +@SuperBuilder +public class WorkflowExecutionDAG implements IWorkflowExecutionDAG { + + private final ITaskExecutionContextFactory taskExecutionContextFactory; + + private final ITaskExecutionRunnableFactory taskExecutionRunnableFactory; + + private final TaskExecutionRunnableRepository taskExecutionRunnableRepository; + + private final IWorkflowDAG workflowDAG; + + @Override + public TaskExecutionRunnable triggerTask(String taskName) { + DAGNode dagNode = getDAGNode(taskName); + if (dagNode == null) { + throw new IllegalArgumentException("Cannot find the DAGNode for task: " + taskName); + } + // Create task execution context + taskExecutionContextFactory.createTaskExecutionContext(); + TaskExecutionRunnable taskExecutionRunnable = taskExecutionRunnableFactory.createTaskExecutionRunnable(null); + taskExecutionRunnableRepository.storeTaskExecutionRunnable(taskExecutionRunnable); + return taskExecutionRunnable; + } + + @Override + public TaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId) { + return taskExecutionRunnableRepository.getTaskExecutionRunnableById(taskInstanceId); + } + + @Override + public TaskExecutionRunnable getTaskExecutionRunnableByName(String taskName) { + return taskExecutionRunnableRepository.getTaskExecutionRunnableByName(taskName); + } + + @Override + public List getActiveTaskExecutionRunnable() { + return new ArrayList<>(taskExecutionRunnableRepository.getActiveTaskExecutionRunnable()); + } + + @Override + public List getDirectPreTaskExecutionRunnable(String taskName) { + return getDirectPreNodeNames(taskName) + .stream() + .map(taskExecutionRunnableRepository::getTaskExecutionRunnableByName) + .collect(Collectors.toList()); + } + + @Override + public boolean isTaskAbleToBeTriggered(String taskNodeName) { + // todo: Check whether the workflow instance is finished or ready to finish. + List directPreNodes = getDirectPreNodes(taskNodeName); + if (log.isDebugEnabled()) { + log.debug("Begin to check whether the task {} is able to be triggered.", taskNodeName); + log.debug("Task {} directly dependent on the task: {}.", taskNodeName, + directPreNodes.stream().map(DAGNode::getNodeName).collect(Collectors.toList())); + } + for (DAGNode directPreNode : directPreNodes) { + if (directPreNode.isSkip()) { + log.debug("The task {} is skipped.", directPreNode.getNodeName()); + continue; + } + TaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnableByName(directPreNode.getNodeName()); + if (taskExecutionRunnable == null || taskExecutionRunnable.canAccessTo(taskNodeName)) { + log.debug("The task {} is not finished or not able to access to the task {}.", + directPreNode.getNodeName(), taskNodeName); + } + } + return true; + } + + @Override + public List getDirectPostNodes(DAGNode dagNode) { + return workflowDAG.getDirectPostNodes(dagNode); + } + + @Override + public List getDirectPreNodes(DAGNode dagNode) { + return workflowDAG.getDirectPreNodes(dagNode); + } + + @Override + public DAGNode getDAGNode(String nodeName) { + return workflowDAG.getDAGNode(nodeName); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAGFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAGFactory.java new file mode 100644 index 0000000000..fa1e897c20 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAGFactory.java @@ -0,0 +1,60 @@ +/* + * 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.dag; + +import java.util.ArrayList; +import java.util.List; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionDAGFactory implements IWorkflowExecutionDAGFactory { + + @Autowired + private ITaskExecutionRunnableRepositoryFactory taskExecutionRunnableRepositoryFactory; + + @Autowired + private ITaskExecutionRunnableFactory taskExecutionRunnableFactory; + + @Autowired + private ITaskExecutionContextFactory taskExecutionContextFactory; + + @Override + public IWorkflowExecutionDAG createWorkflowExecutionDAG() { + // todo: + TaskExecutionRunnableRepository taskExecutionRunnableRepository = + taskExecutionRunnableRepositoryFactory.createTaskExecutionRunnableRepository(); + loadTheHistoryTaskExecutionRunnable() + .forEach(taskExecutionRunnableRepository::storeTaskExecutionRunnable); + + return WorkflowExecutionDAG.builder() + .taskExecutionContextFactory(taskExecutionContextFactory) + .taskExecutionRunnableFactory(taskExecutionRunnableFactory) + .taskExecutionRunnableRepository(taskExecutionRunnableRepository) + .workflowDAG(null) + .build(); + } + + private List loadTheHistoryTaskExecutionRunnable() { + return new ArrayList<>(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnable.java new file mode 100644 index 0000000000..1c9b4208e7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnable.java @@ -0,0 +1,64 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.server.master.events.IEventRepository; +import org.apache.dolphinscheduler.server.master.events.WorkflowTriggeredEvent; + +import org.apache.commons.collections4.CollectionUtils; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@Getter +@Builder +@AllArgsConstructor +public class WorkflowExecutionRunnable implements IWorkflowExecutionRunnable { + + private final IWorkflowExecutionContext workflowExecutionContext; + + private final IDAGEngine dagEngine; + + public void start() { + if (CollectionUtils.isEmpty(workflowExecutionContext.getBeginNodeNames())) { + dagEngine.triggerNextTasks(null); + } else { + workflowExecutionContext.getBeginNodeNames().forEach(dagEngine::triggerTask); + } + getEventRepository() + .storeEventToTail(new WorkflowTriggeredEvent(workflowExecutionContext.getWorkflowInstance().getId())); + } + + @Override + public void pause() { + dagEngine.pauseAllTask(); + } + + @Override + public void kill() { + dagEngine.killAllTask(); + } + + @Override + public IEventRepository getEventRepository() { + return workflowExecutionContext.getEventRepository(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableFactory.java new file mode 100644 index 0000000000..4d3f766725 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableFactory.java @@ -0,0 +1,40 @@ +/* + * 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.dag; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecutionRunnableFactory { + + @Autowired + private IDAGEngineFactory dagEngineFactory; + + public WorkflowExecutionRunnable createWorkflowExecuteRunnable(IWorkflowExecutionContext workflowExecutionContext) { + IDAGEngine dagEngine = dagEngineFactory.createDAGEngine(workflowExecutionContext); + return WorkflowExecutionRunnable.builder() + .workflowExecutionContext(workflowExecutionContext) + .dagEngine(dagEngine) + .build(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowIdentify.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowIdentify.java new file mode 100644 index 0000000000..0b7981199a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowIdentify.java @@ -0,0 +1,35 @@ +/* + * 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.dag; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowIdentify { + + private long workflowCode; + + private int workflowVersion; + +} 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 0000000000..f614a59caf --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java @@ -0,0 +1,111 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; + +import org.apache.commons.lang3.time.StopWatch; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class EventEngine extends BaseDaemonThread { + + @Autowired + private WorkflowExecuteRunnableRepository 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.getActiveWorkflowExecutionRunnable(); + for (IWorkflowExecutionRunnable workflowExecutionRunnable : workflowExecutionRunnableCollection) { + IWorkflowExecutionContext workflowExecutionContext = + workflowExecutionRunnable.getWorkflowExecutionContext(); + final Integer workflowInstanceId = workflowExecutionContext.getWorkflowInstanceId(); + final String workflowInstanceName = workflowExecutionContext.getWorkflowInstanceName(); + 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 { + if (fireCount > 0) { + 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/EventFirer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java new file mode 100644 index 0000000000..fbbc683097 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java @@ -0,0 +1,95 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadPoolExecutor; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class EventFirer implements IEventFirer { + + private final IEventOperatorManager eventOperatorManager; + + private final ThreadPoolExecutor eventFireThreadPool; + + public EventFirer(IEventOperatorManager eventOperatorManager, MasterConfig masterConfig) { + this.eventOperatorManager = eventOperatorManager; + this.eventFireThreadPool = + ThreadUtils.newDaemonFixedThreadExecutor("EventFireThreadPool", masterConfig.getExecThreads()); + } + + @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); + } + + 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).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 0000000000..7a7b52d6c6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java @@ -0,0 +1,36 @@ +/* + * 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.events; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +/** + * The event operator manager interface used to get {@link ITaskEventOperator}. + */ +@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/IAsyncEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java new file mode 100644 index 0000000000..0d0395de6e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java @@ -0,0 +1,24 @@ +/* + * 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.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/IDelayEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IDelayEvent.java new file mode 100644 index 0000000000..c86ea4744f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IDelayEvent.java @@ -0,0 +1,26 @@ +/* + * 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.events; + +public interface IDelayEvent { + + long getEventCreateTime(); + + long getDelayTime(); + +} 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 0000000000..0535bbe2c7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java @@ -0,0 +1,22 @@ +/* + * 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.events; + +public interface IEvent { + +} 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 0000000000..f33db8ed6a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java @@ -0,0 +1,35 @@ +/* + * 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.events; + +import java.util.concurrent.CompletableFuture; + +/** + * The event firer interface used to fire event. + * + */ +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 0000000000..23694a1ccb --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java @@ -0,0 +1,32 @@ +/* + * 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.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 0000000000..7fb2b63d8a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java @@ -0,0 +1,33 @@ +/* + * 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.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 0000000000..20e5081124 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java @@ -0,0 +1,37 @@ +/* + * 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.events; + +import java.util.List; + +/** + * The event repository interface used to store event. + */ +public interface IEventRepository { + + void storeEventToTail(IEvent event); + + void storeEventToHead(IEvent event); + + IEvent poolEvent(); + + int getEventSize(); + + List getAllEvent(); + +} 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 0000000000..5a385cc034 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java @@ -0,0 +1,21 @@ +/* + * 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.events; + +public interface ISyncEvent { +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEvent.java new file mode 100644 index 0000000000..5d105df288 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEvent.java @@ -0,0 +1,26 @@ +/* + * 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.events; + +public interface ITaskEvent extends IEvent { + + Integer getWorkflowInstanceId(); + + Integer getTaskInstanceId(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEventOperator.java new file mode 100644 index 0000000000..1aa1c28299 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEventOperator.java @@ -0,0 +1,22 @@ +/* + * 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.events; + +public interface ITaskEventOperator extends IEventOperator { + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEvent.java new file mode 100644 index 0000000000..7b5ae1cd66 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEvent.java @@ -0,0 +1,29 @@ +/* + * 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.events; + +public interface IWorkflowEvent extends IEvent { + + /** + * The id of WorkflowInstance which the event is related to + * + * @return workflowInstanceId, shouldn't be null + */ + Integer getWorkflowInstanceId(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEventOperator.java new file mode 100644 index 0000000000..c601cc27d3 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEventOperator.java @@ -0,0 +1,23 @@ +/* + * 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.events; + +public interface IWorkflowEventOperator + extends + IEventOperator { +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/MemoryEventRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/MemoryEventRepository.java new file mode 100644 index 0000000000..376b2e4597 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/MemoryEventRepository.java @@ -0,0 +1,60 @@ +/* + * 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.events; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class MemoryEventRepository implements IEventRepository { + + protected final LinkedBlockingDeque eventQueue; + + public MemoryEventRepository() { + this.eventQueue = new LinkedBlockingDeque<>(); + } + + @Override + public void storeEventToTail(IEvent event) { + log.info("Store event to tail: {}", event); + eventQueue.offerLast(event); + } + + @Override + public void storeEventToHead(IEvent event) { + eventQueue.offerFirst(event); + } + + @Override + public IEvent poolEvent() { + return eventQueue.poll(); + } + + @Override + public int getEventSize() { + return eventQueue.size(); + } + + public List getAllEvent() { + return new ArrayList<>(eventQueue); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEvent.java new file mode 100644 index 0000000000..5e42a40bd6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEvent.java @@ -0,0 +1,36 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class TaskLogSendToRemoteEvent implements ITaskEvent { + + private Integer workflowInstanceId; + private Integer taskInstanceId; + + private String taskType; + private String logPath; +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEventOperator.java new file mode 100644 index 0000000000..c75f1fcba5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEventOperator.java @@ -0,0 +1,38 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils; +import org.apache.dolphinscheduler.server.master.utils.TaskUtils; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskLogSendToRemoteEventOperator implements ITaskEventOperator { + + @Override + public void handleEvent(TaskLogSendToRemoteEvent event) { + if (RemoteLogUtils.isRemoteLoggingEnable() && TaskUtils.isMasterTask(event.getTaskType())) { + RemoteLogUtils.sendRemoteLog(event.getLogPath()); + log.info("Master sends task log {} to remote storage asynchronously.", event.getLogPath()); + } + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEvent.java new file mode 100644 index 0000000000..165ea88053 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEvent.java @@ -0,0 +1,46 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.server.master.dag.TaskExecutionRunnable; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class TaskOperationEvent implements ITaskEvent, ISyncEvent { + + private TaskExecutionRunnable taskExecutionRunnable; + + private TaskOperationType taskOperationType; + + @Override + public Integer getWorkflowInstanceId() { + return taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getProcessInstanceId(); + } + + @Override + public Integer getTaskInstanceId() { + return taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEventOperator.java new file mode 100644 index 0000000000..355d28085b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEventOperator.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.dag.TaskExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskOperationEventOperator implements ITaskEventOperator { + + @Override + public void handleEvent(TaskOperationEvent event) { + TaskExecutionRunnable taskExecutionRunnable = event.getTaskExecutionRunnable(); + switch (event.getTaskOperationType()) { + case RUN: + taskExecutionRunnable.dispatch(); + break; + case KILL: + taskExecutionRunnable.kill(); + break; + case PAUSE: + taskExecutionRunnable.pause(); + break; + default: + log.error("Unknown TaskOperationType for event: {}", event); + } + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationType.java new file mode 100644 index 0000000000..ae72a03b01 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationType.java @@ -0,0 +1,29 @@ +/* + * 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.events; + +public enum TaskOperationType { + + FAILOVER, + RUN, + RETRY, + KILL, + PAUSE, + ; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEvent.java new file mode 100644 index 0000000000..a84d89069a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEvent.java @@ -0,0 +1,33 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +@AllArgsConstructor +public class TaskSuccessEvent implements ITaskEvent { + + private Integer workflowInstanceId; + + private Integer taskInstanceId; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEventOperator.java new file mode 100644 index 0000000000..8ed9c2033b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEventOperator.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskSuccessEventOperator implements ITaskEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(TaskSuccessEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + Integer taskInstanceId = event.getTaskInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecutionRunnable == null) { + log.error("Cannot find the WorkflowExecutionRunnable, the event: {} will be dropped", event); + return; + } + + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEvent.java new file mode 100644 index 0000000000..f07067c261 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEvent.java @@ -0,0 +1,35 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowFailedEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + + private String failedReason; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEventOperator.java new file mode 100644 index 0000000000..b7e562b6f7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEventOperator.java @@ -0,0 +1,57 @@ +/* + * 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.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.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowFailedEventOperator + implements + IWorkflowEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Override + public void handleEvent(WorkflowFailedEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + workflowInstance.setState(WorkflowExecutionStatus.FAILURE); + processInstanceDao.updateById(workflowInstance); + log.info("Handle WorkflowExecutionRunnableFailedEvent success, set workflowInstance status to {}", + workflowInstance.getState()); + workflowExecutionRunnable.storeEventToTail(new WorkflowFinalizeEvent(workflowInstanceId)); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEvent.java new file mode 100644 index 0000000000..454b755c42 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEvent.java @@ -0,0 +1,32 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@AllArgsConstructor +@NoArgsConstructor +public class WorkflowFinalizeEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEventOperator.java new file mode 100644 index 0000000000..aeab8ae30c --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEventOperator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowFinalizeEventOperator + implements + IWorkflowEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowFinalizeEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + workflowExecuteRunnableRepository.removeWorkflowExecutionRunnable(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinishEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinishEvent.java new file mode 100644 index 0000000000..b58db0cc53 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinishEvent.java @@ -0,0 +1,37 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowFinishEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + + private WorkflowExecutionStatus workflowExecutionStatus; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEvent.java new file mode 100644 index 0000000000..6450067f60 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEvent.java @@ -0,0 +1,51 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +@AllArgsConstructor +public class WorkflowOperationEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + private WorkflowOperationType workflowOperationType; + + public static WorkflowOperationEvent of(Integer workflowInstanceId, WorkflowOperationType workflowOperationType) { + return WorkflowOperationEvent.builder() + .workflowInstanceId(workflowInstanceId) + .workflowOperationType(workflowOperationType) + .build(); + } + + public static WorkflowOperationEvent triggerEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.TRIGGER); + } + + public static WorkflowOperationEvent pauseEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.PAUSE); + } + + public static WorkflowOperationEvent killEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.KILL); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventOperator.java new file mode 100644 index 0000000000..2c8159f88f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventOperator.java @@ -0,0 +1,87 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; +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 WorkflowOperationEventOperator implements IWorkflowEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowOperationEvent event) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(event.getWorkflowInstanceId()); + if (workflowExecutionRunnable == null) { + log.warn( + "Handle workflowExecutionRunnableKillOperationEvent: {} failed: WorkflowExecutionRunnable not found", + event); + return; + } + switch (event.getWorkflowOperationType()) { + case TRIGGER: + triggerWorkflow(workflowExecutionRunnable); + break; + case PAUSE: + pauseWorkflow(workflowExecutionRunnable); + break; + case KILL: + killWorkflow(workflowExecutionRunnable); + break; + default: + log.error("Unknown operationType for event: {}", event); + } + } + + private void triggerWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + try { + workflowExecutionRunnable.start(); + } catch (Throwable exception) { + if (ExceptionUtils.isDatabaseConnectedFailedException(exception)) { + throw exception; + } + IWorkflowExecutionContext workflowExecutionContext = + workflowExecutionRunnable.getWorkflowExecutionContext(); + log.error("Trigger workflow: {} failed", workflowExecutionContext.getWorkflowInstanceName(), exception); + WorkflowFailedEvent workflowExecutionRunnableFailedEvent = WorkflowFailedEvent.builder() + .workflowInstanceId(workflowExecutionContext.getWorkflowInstanceId()) + .failedReason(exception.getMessage()) + .build(); + workflowExecutionRunnable.storeEventToTail(workflowExecutionRunnableFailedEvent); + } + } + + private void pauseWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnable.pause(); + } + + private void killWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnable.kill(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationType.java new file mode 100644 index 0000000000..24abbbc855 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationType.java @@ -0,0 +1,36 @@ +/* + * 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.events; + +public enum WorkflowOperationType { + + /** + * Trigger the workflow instance. + */ + TRIGGER, + /** + * Pause the workflow instance, it will pause the running task instances. + */ + PAUSE, + /** + * Kill the workflow instance, it will kill the running task instances. + */ + KILL, + ; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEvent.java new file mode 100644 index 0000000000..b075d194d7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEvent.java @@ -0,0 +1,33 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowTimeoutEvent implements IWorkflowEvent, IAsyncEvent { + + private Integer workflowInstanceId; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEventOperator.java new file mode 100644 index 0000000000..ebb43e3359 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEventOperator.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProjectUser; +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; +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 WorkflowTimeoutEventOperator + implements + IEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private ProcessService processService; + + @Autowired + private ProcessAlertManager processAlertManager; + + @Override + public void handleEvent(WorkflowTimeoutEvent event) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(event.getWorkflowInstanceId()); + if (workflowExecutionRunnable == null) { + log.warn("Cannot find the workflow instance by id: {}", event.getWorkflowInstanceId()); + return; + } + // we only support timeout warning for now + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(workflowInstance.getId()); + processAlertManager.sendProcessTimeoutAlert(workflowInstance, projectUser); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEvent.java new file mode 100644 index 0000000000..3dfdb9e239 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEvent.java @@ -0,0 +1,38 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowTriggerNextTaskEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + + /** + * The task name of the parent task, if it is the root task, the value is null + */ + private String parentTaskNodeName; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEventOperator.java new file mode 100644 index 0000000000..d294b84d58 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEventOperator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.events; + +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowTriggerNextTaskEventOperator + implements + IWorkflowEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Override + public void handleEvent(WorkflowTriggerNextTaskEvent event) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(event.getWorkflowInstanceId()); + workflowExecutionRunnable.getDagEngine().triggerNextTasks(event.getParentTaskNodeName()); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEvent.java new file mode 100644 index 0000000000..b07434dc29 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEvent.java @@ -0,0 +1,33 @@ +/* + * 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.events; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowTriggeredEvent implements IWorkflowEvent, IAsyncEvent { + + private Integer workflowInstanceId; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEventOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEventOperator.java new file mode 100644 index 0000000000..e49b1ae70e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEventOperator.java @@ -0,0 +1,63 @@ +/* + * 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.events; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProjectUser; +import org.apache.dolphinscheduler.server.master.dag.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.server.master.dag.WorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; +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 WorkflowTriggeredEventOperator + implements + IWorkflowEventOperator { + + @Autowired + private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + @Autowired + private ProcessService processService; + + @Autowired + private ListenerEventAlertManager listenerEventAlertManager; + + @Override + public void handleEvent(WorkflowTriggeredEvent event) { + int workflowInstanceId = event.getWorkflowInstanceId(); + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + Long workflowDefinitionCode = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance() + .getProcessDefinitionCode(); + ProcessInstanceMetrics.incProcessInstanceByStateAndProcessDefinitionCode("submit", "" + workflowDefinitionCode); + + ProcessInstance workflowInstance = + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance(); + ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(workflowInstanceId); + 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 0000000000..70088d4970 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java @@ -0,0 +1,29 @@ +/* + * 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 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/WorkflowExecuteRunnableNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java new file mode 100644 index 0000000000..f00a8d36e9 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java @@ -0,0 +1,30 @@ +/* + * 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 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/runner/DefaultTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecuteRunnable.java index c1b13717bd..a1e60e0a6e 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/DefaultTaskExecuteRunnable.java @@ -28,8 +28,7 @@ public class DefaultTaskExecuteRunnable extends PriorityDelayTaskExecuteRunnable private final TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; - public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance, - TaskInstance taskInstance, + public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance, TaskInstance taskInstance, TaskExecutionContext taskExecutionContext, TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager) { super(workflowInstance, taskInstance, taskExecutionContext); 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 ab1806ff67..742c8187d1 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.api.utils.MapUtils; 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/execute/DefaultTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java index ab749b5861..6c1a4e8569 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 @@ -19,8 +19,6 @@ 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.TaskExecuteRunnableFactory; import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory; @@ -46,17 +44,13 @@ public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFac private TaskExecuteRunnableOperatorManager taskExecuteRunnableOperatorManager; @Override - public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException { + public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) { WorkflowExecuteRunnable workflowExecuteRunnable = processInstanceExecCacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId()); - try { - return new DefaultTaskExecuteRunnable( - workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(), - taskInstance, - taskExecutionContextFactory.createTaskExecutionContext(taskInstance), - taskExecuteRunnableOperatorManager); - } catch (TaskExecutionContextCreateException ex) { - throw new TaskExecuteRunnableCreateException("Create DefaultTaskExecuteRunnable failed", ex); - } + return new DefaultTaskExecuteRunnable( + workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowInstance(), + taskInstance, + taskExecutionContextFactory.createTaskExecutionContext(taskInstance), + taskExecuteRunnableOperatorManager); } } 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 0000000000..7cb69b23f5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.utils; + +import 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/test/java/org/apache/dolphinscheduler/server/master/dag/DAGNodeTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/DAGNodeTest.java new file mode 100644 index 0000000000..c3346bfc5d --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/DAGNodeTest.java @@ -0,0 +1,121 @@ +/* + * 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.dag; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.ArrayList; + +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +class DAGNodeTest { + + @Test + void buildDAGNode_EmptyNodeName() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build()); + assertEquals("nodeName cannot be empty", illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_BadInDegree() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .nodeName("A") + .inDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName(null) + .toNodeName("B") + .build())) + .outDegrees(new ArrayList<>()) + .build()); + assertEquals( + "The toNodeName of inDegree should be the nodeName of the node: A, inDegree: DAGEdge(fromNodeName=null, toNodeName=B)", + illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_NiceInDegree() { + assertDoesNotThrow(() -> DAGNode.builder() + .nodeName("A") + .inDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName(null) + .toNodeName("A") + .build())) + .outDegrees(new ArrayList<>()) + .build()); + } + + @Test + void buildDAGNode_BadOutDegree() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName("B") + .toNodeName(null) + .build())) + .build()); + assertEquals( + "The fromNodeName of outDegree should be the nodeName of the node: A, outDegree: DAGEdge(fromNodeName=B, toNodeName=null)", + illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_NiceOutDegree() { + assertDoesNotThrow(() -> DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName("A") + .toNodeName(null) + .build())) + .build()); + } + + @Test + void buildDAGNode_NotSkip() { + DAGNode dagNode = DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build(); + assertFalse(dagNode.isSkip()); + } + + @Test + void buildDAGNode_Skip() { + DAGNode dagNode = DAGNode.builder() + .nodeName("A") + .skip(true) + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build(); + assertTrue(dagNode.isSkip()); + } + +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionContextFactory.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionContextFactory.java new file mode 100644 index 0000000000..b067a455dd --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionContextFactory.java @@ -0,0 +1,88 @@ +/* + * 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.dag; + +import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog; +import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog; +import org.apache.dolphinscheduler.server.master.events.MemoryEventRepository; + +import java.util.Collections; + +class MockWorkflowExecutionContextFactory { + + public static WorkflowExecutionContext createWorkflowExecutionContext() { + long workflowDefinitionCode = CodeGenerateUtils.getInstance().genCode(); + int workflowDefinitionVersion = 1; + + int workflowInstanceId = 1; + String workflowInstanceName = "MockWorkflow" + CodeGenerateUtils.getInstance().genCode(); + + ProcessDefinition processDefinition = ProcessDefinition.builder() + .id(1) + .name("TestWorkflow") + .code(workflowDefinitionCode) + .version(workflowDefinitionVersion) + .build(); + + ProcessInstance processInstance = ProcessInstance.builder() + .id(workflowInstanceId) + .name(workflowInstanceName) + .build(); + + WorkflowExecutionDAG workflowExecutionDAG = createWorkflowExecutionDAG(); + + return WorkflowExecutionContext.builder() + .workflowExecutionDAG(workflowExecutionDAG) + .eventRepository(new MemoryEventRepository()) + .beginNodeNames(Collections.emptyList()) + .workflowDefinition(processDefinition) + .workflowInstance(processInstance) + .build(); + } + + private static WorkflowExecutionDAG createWorkflowExecutionDAG() { + TaskDefinitionLog taskA = taskNode("A"); + WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder() + .addTaskNode(taskA) + .addTaskEdge(edge(null, taskA)) + .addTaskEdge(edge(taskA, null)) + .build(); + + return WorkflowExecutionDAG.builder() + .workflowDAG(workflowDAG) + .taskExecutionRunnableRepository(new TaskExecutionRunnableRepository()) + .build(); + } + + private static TaskDefinitionLog taskNode(String nodeName) { + TaskDefinitionLog taskDefinitionLog = new TaskDefinitionLog(); + taskDefinitionLog.setCode(CodeGenerateUtils.getInstance().genCode()); + taskDefinitionLog.setName(nodeName); + return taskDefinitionLog; + } + + private static ProcessTaskRelationLog edge(TaskDefinitionLog from, TaskDefinitionLog to) { + ProcessTaskRelationLog processTaskRelationLog = new ProcessTaskRelationLog(); + processTaskRelationLog.setPreTaskCode(from == null ? 0 : from.getCode()); + processTaskRelationLog.setPostTaskCode(to == null ? 0 : to.getCode()); + return processTaskRelationLog; + } +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionRunnableFactory.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionRunnableFactory.java new file mode 100644 index 0000000000..77745e198b --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionRunnableFactory.java @@ -0,0 +1,31 @@ +/* + * 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.dag; + +class MockWorkflowExecutionRunnableFactory { + + public static WorkflowExecutionRunnable createWorkflowExecuteRunnable() { + WorkflowExecutionContext workflowExecutionContext = + MockWorkflowExecutionContextFactory.createWorkflowExecutionContext(); + IDAGEngine dagEngine = new DAGEngine(workflowExecutionContext); + return WorkflowExecutionRunnable.builder() + .workflowExecutionContext(workflowExecutionContext) + .dagEngine(dagEngine) + .build(); + } +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGAssertion.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGAssertion.java new file mode 100644 index 0000000000..a03ecd9f76 --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGAssertion.java @@ -0,0 +1,171 @@ +/* + * 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.dag; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.apache.commons.collections4.CollectionUtils; + +import java.util.ArrayList; +import java.util.List; + +public class WorkflowDAGAssertion { + + private final IWorkflowDAG workflowDAG; + + private final List nodeAssertions; + + private WorkflowDAGAssertion(IWorkflowDAG workflowDAG) { + this.workflowDAG = workflowDAG; + this.nodeAssertions = new ArrayList<>(); + } + + public static WorkflowDAGAssertion workflowDag(IWorkflowDAG workflowDAG) { + return new WorkflowDAGAssertion(workflowDAG); + } + + public WorkflowDAGAssertion nodeAssertion(NodeAssertion nodeAssertion) { + nodeAssertions.add(nodeAssertion); + return this; + } + + public void doAssertion() { + nodeAssertions.forEach(nodeAssertion -> nodeAssertion.doAssertion(workflowDAG)); + } + + public static class NodeAssertion { + + /** + * node name of the assertion + */ + private final String nodeName; + + /** + * whether the node exist + */ + private boolean exist; + + /** + * whether the node is skipped + */ + private boolean skip; + + /** + * whether the node has out degree + */ + private List postNodes; + + /** + * whether the node has in degree + */ + private List preNodes; + + private NodeAssertion(String nodeName) { + this.nodeName = nodeName; + this.postNodes = new ArrayList<>(); + this.preNodes = new ArrayList<>(); + } + + public static NodeAssertion node(String nodeName) { + return new NodeAssertion(nodeName); + } + + public NodeAssertion exist() { + this.exist = true; + return this; + } + + public NodeAssertion skip() { + this.skip = true; + return this; + } + + public NodeAssertion noEdge() { + this.postNodes = new ArrayList<>(); + this.preNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion noInDegree() { + this.preNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion inDegrees(List preNodes) { + this.preNodes.addAll(preNodes); + return this; + } + + public NodeAssertion inDegrees(String preNode) { + this.preNodes.add(preNode); + return this; + } + + public NodeAssertion noOutDegree() { + this.postNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion outDegrees(List postNodes) { + this.postNodes.addAll(postNodes); + return this; + } + + public NodeAssertion outDegrees(String postNode) { + this.postNodes.add(postNode); + return this; + } + + private void doAssertion(IWorkflowDAG workflowDAG) { + if (exist) { + // node exist + assertNotNull(workflowDAG.getDAGNode(nodeName), "node " + nodeName + " does not exist"); + } else { + + assertNull(workflowDAG.getDAGNode(nodeName), "node " + nodeName + " exist"); + } + + if (skip) { + assertTrue(workflowDAG.getDAGNode(nodeName).isSkip(), "node " + nodeName + " is not skipped"); + } else { + assertFalse(workflowDAG.getDAGNode(nodeName).isSkip(), "node " + nodeName + " is skipped"); + } + + if (CollectionUtils.isEmpty(postNodes)) { + assertTrue(workflowDAG.getDirectPostNodeNames(nodeName).isEmpty(), + "node " + nodeName + " has outDegree " + workflowDAG.getDirectPostNodes(nodeName)); + } else { + postNodes + .forEach(postNode -> assertTrue(workflowDAG.getDirectPostNodeNames(nodeName).contains(postNode), + "node " + nodeName + " does has outDegree " + postNode)); + } + + if (CollectionUtils.isEmpty(preNodes)) { + assertTrue(workflowDAG.getDirectPreNodeNames(nodeName).isEmpty(), + "node " + nodeName + " has inDegree " + workflowDAG.getDirectPreNodeNames(nodeName)); + } else { + preNodes.forEach(preNode -> assertTrue(workflowDAG.getDirectPreNodeNames(nodeName).contains(preNode), + "node " + nodeName + " does has inDegree " + preNode)); + } + } + } + +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilderTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilderTest.java new file mode 100644 index 0000000000..a4abb8156f --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilderTest.java @@ -0,0 +1,282 @@ +/* + * 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.dag; + +import static com.google.common.collect.Lists.newArrayList; +import static org.apache.dolphinscheduler.server.master.dag.WorkflowDAGAssertion.NodeAssertion.node; +import static org.apache.dolphinscheduler.server.master.dag.WorkflowDAGAssertion.workflowDag; + +import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog; +import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog; + +import org.junit.jupiter.api.Test; + +class WorkflowDAGBuilderTest { + + /** + * Test DAG with single node: + *
    +     *     {@code
    +     *         Node(A)
    +     *     }
    +     *  
    + */ + @Test + void build_SingleTaskNode() { + String nodeName = "A"; + TaskDefinitionLog taskDefinitionLog = taskNode(nodeName); + WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder() + .addTaskNode(taskDefinitionLog) + .build(); + workflowDag(workflowDAG) + .nodeAssertion(node(nodeName).exist().noEdge()) + .doAssertion(); + } + + /** + * Test DAG with multiple nodes: + *
    +     *     {@code
    +     *        Node(A)
    +     *        Node(B)
    +     *        Node(C)
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_MULTIPLE_NODE() {
    +        String nodeNameA = "A";
    +        TaskDefinitionLog taskNodeA = taskNode(nodeNameA);
    +        String nodeNameB = "B";
    +        TaskDefinitionLog taskNodeB = taskNode(nodeNameB);
    +        String nodeNameC = "C";
    +        TaskDefinitionLog taskNodeC = taskNode(nodeNameC);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNode(taskNodeA)
    +                .addTaskNode(taskNodeB)
    +                .addTaskNode(taskNodeC)
    +                .build();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA).exist().noEdge())
    +                .nodeAssertion(node(nodeNameB).exist().noEdge())
    +                .nodeAssertion(node(nodeNameC).exist().noEdge())
    +                .doAssertion();
    +    }
    +
    +    /**
    +     * Test DAG with multiple nodes:
    +     * 
    +     *     {@code
    +     *          Node(A) -> Node(B1) -> Node(C1) -> Node(D)
    +     *                  -> Node(B2) -> Node(C2) ->
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_DAG() {
    +        String nodeNameA = "A";
    +        TaskDefinitionLog taskNodeA = taskNode(nodeNameA);
    +        String nodeNameB1 = "B1";
    +        TaskDefinitionLog taskNodeB1 = taskNode(nodeNameB1);
    +        String nodeNameB2 = "B2";
    +        TaskDefinitionLog taskNodeB2 = taskNode(nodeNameB2);
    +        String nodeNameC1 = "C1";
    +        TaskDefinitionLog taskNodeC1 = taskNode(nodeNameC1);
    +        String nodeNameC2 = "C2";
    +        TaskDefinitionLog taskNodeC2 = taskNode(nodeNameC2);
    +        String nodeNameD = "D";
    +        TaskDefinitionLog taskNodeD = taskNode(nodeNameD);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNode(taskNodeA)
    +                .addTaskNodes(newArrayList(taskNodeB1, taskNodeB2))
    +                .addTaskNodes(newArrayList(taskNodeC1, taskNodeC2))
    +                .addTaskNode(taskNodeD)
    +                .addTaskEdge(edge(null, taskNodeA))
    +                .addTaskEdge(edge(taskNodeA, taskNodeB1))
    +                .addTaskEdge(edge(taskNodeA, taskNodeB2))
    +                .addTaskEdge(edge(taskNodeB1, taskNodeC1))
    +                .addTaskEdge(edge(taskNodeB2, taskNodeC2))
    +                .addTaskEdge(edge(taskNodeC1, taskNodeD))
    +                .addTaskEdge(edge(taskNodeC2, taskNodeD))
    +                .addTaskEdge(edge(taskNodeD, null))
    +                .build();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB1, nodeNameB2)))
    +                .nodeAssertion(node(nodeNameB1)
    +                        .exist()
    +                        .inDegrees(nodeNameA)
    +                        .outDegrees(nodeNameC1))
    +                .nodeAssertion(node(nodeNameB2)
    +                        .exist()
    +                        .inDegrees(nodeNameA)
    +                        .outDegrees(nodeNameC2))
    +                .nodeAssertion(node(nodeNameC1)
    +                        .exist()
    +                        .inDegrees(nodeNameB1)
    +                        .outDegrees(nodeNameD))
    +                .nodeAssertion(node(nodeNameC2)
    +                        .exist()
    +                        .inDegrees(nodeNameB2)
    +                        .outDegrees(nodeNameD))
    +                .nodeAssertion(node(nodeNameD)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC1, nodeNameC2))
    +                        .noOutDegree())
    +                .doAssertion();
    +    }
    +
    +    /**
    +     * Test DAG with multiple sub dags:
    +     * 
    +     *     {@code
    +     *          Node(A1) -> Node(B1) -> Node(C1) -> Node(D1)
    +     *                  -> Node(B2) -> Node(C2) ->
    +     *
    +     *          Node(A2) -> Node(B3) -> Node(C3) -> Node(D2)
    +     *                   -> Node(B4) -> Node(C4) ->
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_MULTIPLE_SUB_DAG() {
    +        String nodeNameA1 = "A1";
    +        TaskDefinitionLog taskNodeA1 = taskNode(nodeNameA1);
    +        String nodeNameA2 = "A2";
    +        TaskDefinitionLog taskNodeA2 = taskNode(nodeNameA2);
    +
    +        String nodeNameB1 = "B1";
    +        TaskDefinitionLog taskNodeB1 = taskNode(nodeNameB1);
    +        String nodeNameB2 = "B2";
    +        TaskDefinitionLog taskNodeB2 = taskNode(nodeNameB2);
    +        String nodeNameB3 = "B3";
    +        TaskDefinitionLog taskNodeB3 = taskNode(nodeNameB3);
    +        String nodeNameB4 = "B4";
    +        TaskDefinitionLog taskNodeB4 = taskNode(nodeNameB4);
    +
    +        String nodeNameC1 = "C1";
    +        TaskDefinitionLog taskNodeC1 = taskNode(nodeNameC1);
    +        String nodeNameC2 = "C2";
    +        TaskDefinitionLog taskNodeC2 = taskNode(nodeNameC2);
    +        String nodeNameC3 = "C3";
    +        TaskDefinitionLog taskNodeC3 = taskNode(nodeNameC3);
    +        String nodeNameC4 = "C4";
    +        TaskDefinitionLog taskNodeC4 = taskNode(nodeNameC4);
    +
    +        String nodeNameD1 = "D1";
    +        TaskDefinitionLog taskNodeD1 = taskNode(nodeNameD1);
    +        String nodeNameD2 = "D2";
    +        TaskDefinitionLog taskNodeD2 = taskNode(nodeNameD2);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNodes(newArrayList(taskNodeA1, taskNodeA2))
    +                .addTaskNodes(newArrayList(taskNodeB1, taskNodeB2, taskNodeB3, taskNodeB4))
    +                .addTaskNodes(newArrayList(taskNodeC1, taskNodeC2, taskNodeC3, taskNodeC4))
    +                .addTaskNodes(newArrayList(taskNodeD1, taskNodeD2))
    +                .addTaskEdge(edge(null, taskNodeA1))
    +                .addTaskEdge(edge(taskNodeA1, taskNodeB1))
    +                .addTaskEdge(edge(taskNodeA1, taskNodeB2))
    +                .addTaskEdge(edge(taskNodeB1, taskNodeC1))
    +                .addTaskEdge(edge(taskNodeB2, taskNodeC2))
    +                .addTaskEdge(edge(taskNodeC1, taskNodeD1))
    +                .addTaskEdge(edge(taskNodeC2, taskNodeD1))
    +                .addTaskEdge(edge(taskNodeD1, null))
    +                .addTaskEdge(edge(null, taskNodeA2))
    +                .addTaskEdge(edge(taskNodeA2, taskNodeB3))
    +                .addTaskEdge(edge(taskNodeA2, taskNodeB4))
    +                .addTaskEdge(edge(taskNodeB3, taskNodeC3))
    +                .addTaskEdge(edge(taskNodeB4, taskNodeC4))
    +                .addTaskEdge(edge(taskNodeC3, taskNodeD2))
    +                .addTaskEdge(edge(taskNodeC4, taskNodeD2))
    +                .addTaskEdge(edge(taskNodeD2, null))
    +                .build();
    +
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA1)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB1, nodeNameB2)))
    +                .nodeAssertion(node(nodeNameB1)
    +                        .exist()
    +                        .inDegrees(nodeNameA1)
    +                        .outDegrees(nodeNameC1))
    +                .nodeAssertion(node(nodeNameB2)
    +                        .exist()
    +                        .inDegrees(nodeNameA1)
    +                        .outDegrees(nodeNameC2))
    +                .nodeAssertion(node(nodeNameC1)
    +                        .exist()
    +                        .inDegrees(nodeNameB1)
    +                        .outDegrees(nodeNameD1))
    +                .nodeAssertion(node(nodeNameC2)
    +                        .exist()
    +                        .inDegrees(nodeNameB2)
    +                        .outDegrees(nodeNameD1))
    +                .nodeAssertion(node(nodeNameD1)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC1, nodeNameC2))
    +                        .noOutDegree())
    +                .doAssertion();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA2)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB3, nodeNameB4)))
    +                .nodeAssertion(node(nodeNameB3)
    +                        .exist()
    +                        .inDegrees(nodeNameA2)
    +                        .outDegrees(nodeNameC3))
    +                .nodeAssertion(node(nodeNameB4)
    +                        .exist()
    +                        .inDegrees(nodeNameA2)
    +                        .outDegrees(nodeNameC4))
    +                .nodeAssertion(node(nodeNameC3)
    +                        .exist()
    +                        .inDegrees(nodeNameB3)
    +                        .outDegrees(nodeNameD2))
    +                .nodeAssertion(node(nodeNameC4)
    +                        .exist()
    +                        .inDegrees(nodeNameB4)
    +                        .outDegrees(nodeNameD2))
    +                .nodeAssertion(node(nodeNameD2)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC3, nodeNameC4))
    +                        .noOutDegree())
    +                .doAssertion();
    +    }
    +
    +    private TaskDefinitionLog taskNode(String nodeName) {
    +        TaskDefinitionLog taskDefinitionLog = new TaskDefinitionLog();
    +        taskDefinitionLog.setCode(CodeGenerateUtils.getInstance().genCode());
    +        taskDefinitionLog.setName(nodeName);
    +        return taskDefinitionLog;
    +    }
    +
    +    private ProcessTaskRelationLog edge(TaskDefinitionLog from, TaskDefinitionLog to) {
    +        ProcessTaskRelationLog processTaskRelationLog = new ProcessTaskRelationLog();
    +        processTaskRelationLog.setPreTaskCode(from == null ? 0 : from.getCode());
    +        processTaskRelationLog.setPostTaskCode(to == null ? 0 : to.getCode());
    +        return processTaskRelationLog;
    +    }
    +
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineIT.java
    new file mode 100644
    index 0000000000..ede8b55a37
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineIT.java
    @@ -0,0 +1,22 @@
    +/*
    + * 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.dag;
    +
    +public class WorkflowEngineIT {
    +
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineTest.java
    new file mode 100644
    index 0000000000..b6e5dc9db5
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineTest.java
    @@ -0,0 +1,114 @@
    +/*
    + * 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.dag;
    +
    +import static org.apache.dolphinscheduler.server.master.dag.WorkflowExecutionRunnableAssertions.workflowExecutionRunnable;
    +import static org.apache.dolphinscheduler.server.master.dag.WorkflowExecutionRunnableRepositoryAssertions.workflowExecutionRunnableRepository;
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertThrows;
    +
    +import org.apache.dolphinscheduler.server.master.events.WorkflowOperationEvent;
    +import org.apache.dolphinscheduler.server.master.exception.WorkflowExecuteRunnableNotFoundException;
    +
    +import org.junit.jupiter.api.BeforeEach;
    +import org.junit.jupiter.api.Test;
    +
    +class WorkflowEngineTest {
    +
    +    private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository;
    +    private WorkflowEngine workflowEngine;
    +
    +    @BeforeEach
    +    public void before() {
    +        workflowExecuteRunnableRepository = new WorkflowExecuteRunnableRepository();
    +        workflowEngine = new WorkflowEngine(workflowExecuteRunnableRepository);
    +    }
    +
    +    @Test
    +    void triggerWorkflow() {
    +        IWorkflowExecutionRunnable emptyWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecuteRunnable();
    +        Integer workflowInstanceId = emptyWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +
    +        workflowEngine.triggerWorkflow(emptyWorkflowExecuteRunnable);
    +        workflowExecutionRunnable(emptyWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.triggerEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void pauseWorkflow_WorkflowNotExist() {
    +        WorkflowExecuteRunnableNotFoundException exception =
    +                assertThrows(WorkflowExecuteRunnableNotFoundException.class, () -> workflowEngine.pauseWorkflow(1));
    +        assertEquals("WorkflowExecuteRunnable not found: [id=1]", exception.getMessage());
    +    }
    +
    +    @Test
    +    void pauseWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable emptyWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecuteRunnable();
    +        Integer workflowInstanceId =
    +                emptyWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        workflowExecuteRunnableRepository.storeWorkflowExecutionRunnable(emptyWorkflowExecuteRunnable);
    +
    +        workflowEngine.pauseWorkflow(workflowInstanceId);
    +        workflowExecutionRunnable(emptyWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.pauseEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void killWorkflow_WorkflowNotExist() {
    +        WorkflowExecuteRunnableNotFoundException exception =
    +                assertThrows(WorkflowExecuteRunnableNotFoundException.class,
    +                        () -> workflowEngine.killWorkflow(1));
    +        assertEquals("WorkflowExecuteRunnable not found: [id=1]", exception.getMessage());
    +    }
    +
    +    @Test
    +    void killWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable emptyWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecuteRunnable();
    +        Integer workflowInstanceId =
    +                emptyWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        workflowExecuteRunnableRepository.storeWorkflowExecutionRunnable(emptyWorkflowExecuteRunnable);
    +
    +        workflowEngine.killWorkflow(workflowInstanceId);
    +        workflowExecutionRunnable(emptyWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.killEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void finalizeWorkflow_WorkflowNotExist() {
    +        workflowEngine.finalizeWorkflow(1);
    +    }
    +
    +    @Test
    +    void finalizeWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable emptyWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecuteRunnable();
    +        Integer workflowInstanceId =
    +                emptyWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        workflowExecuteRunnableRepository.storeWorkflowExecutionRunnable(emptyWorkflowExecuteRunnable);
    +        workflowExecutionRunnableRepository(workflowExecuteRunnableRepository)
    +                .existWorkflowExecutionRunnable(workflowInstanceId);
    +
    +        workflowEngine.finalizeWorkflow(workflowInstanceId);
    +        workflowExecutionRunnableRepository(workflowExecuteRunnableRepository)
    +                .notExistWorkflowExecutionRunnable(workflowInstanceId);
    +    }
    +
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableAssertions.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableAssertions.java
    new file mode 100644
    index 0000000000..65cf7ba7cd
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableAssertions.java
    @@ -0,0 +1,48 @@
    +/*
    + * 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.dag;
    +
    +import org.apache.dolphinscheduler.server.master.events.IEvent;
    +
    +import org.opentest4j.AssertionFailedError;
    +
    +class WorkflowExecutionRunnableAssertions {
    +
    +    private final IWorkflowExecutionRunnable workflowExecutionRunnable;
    +
    +    private WorkflowExecutionRunnableAssertions(IWorkflowExecutionRunnable workflowExecutionRunnable) {
    +        this.workflowExecutionRunnable = workflowExecutionRunnable;
    +    }
    +
    +    public static WorkflowExecutionRunnableAssertions workflowExecutionRunnable(IWorkflowExecutionRunnable workflowExecutionRunnable) {
    +        return new WorkflowExecutionRunnableAssertions(workflowExecutionRunnable);
    +    }
    +
    +    public void existEvent(IEvent event) {
    +        if (event == null) {
    +            throw new IllegalArgumentException("Event cannot be null");
    +        }
    +        boolean exist = workflowExecutionRunnable.getEventRepository().getAllEvent()
    +                .stream()
    +                .anyMatch(event1 -> event1.equals(event1));
    +        if (!exist) {
    +            throw new AssertionFailedError("The workflowExecuteRunnable doesn't exist event: " + event);
    +        }
    +    }
    +
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableRepositoryAssertions.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableRepositoryAssertions.java
    new file mode 100644
    index 0000000000..e979f35cb3
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableRepositoryAssertions.java
    @@ -0,0 +1,43 @@
    +/*
    + * 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.dag;
    +
    +import static org.junit.jupiter.api.Assertions.assertNotNull;
    +import static org.junit.jupiter.api.Assertions.assertNull;
    +
    +class WorkflowExecutionRunnableRepositoryAssertions {
    +
    +    private final WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository;
    +
    +    private WorkflowExecutionRunnableRepositoryAssertions(WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) {
    +        this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository;
    +    }
    +
    +    public static WorkflowExecutionRunnableRepositoryAssertions workflowExecutionRunnableRepository(WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) {
    +        return new WorkflowExecutionRunnableRepositoryAssertions(workflowExecuteRunnableRepository);
    +    }
    +
    +    public void existWorkflowExecutionRunnable(Integer workflowInstanceId) {
    +        assertNotNull(workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId));
    +    }
    +
    +    public void notExistWorkflowExecutionRunnable(Integer workflowInstanceId) {
    +        assertNull(workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId));
    +    }
    +
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableTest.java
    new file mode 100644
    index 0000000000..69a7f83a50
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableTest.java
    @@ -0,0 +1,51 @@
    +/*
    + * 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.dag;
    +
    +import static org.apache.dolphinscheduler.server.master.dag.WorkflowExecutionRunnableAssertions.workflowExecutionRunnable;
    +
    +import org.apache.dolphinscheduler.server.master.events.WorkflowTriggeredEvent;
    +
    +import org.junit.jupiter.api.BeforeEach;
    +import org.junit.jupiter.api.Test;
    +
    +class WorkflowExecutionRunnableTest {
    +
    +    private WorkflowExecutionRunnable workflowExecutionRunnable;
    +
    +    @BeforeEach
    +    public void before() {
    +        workflowExecutionRunnable = MockWorkflowExecutionRunnableFactory.createWorkflowExecuteRunnable();
    +    }
    +
    +    @Test
    +    void start() {
    +        workflowExecutionRunnable.start();
    +        workflowExecutionRunnable(workflowExecutionRunnable)
    +                .existEvent(new WorkflowTriggeredEvent(
    +                        workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstance().getId()));
    +    }
    +
    +    @Test
    +    void pause() {
    +    }
    +
    +    @Test
    +    void kill() {
    +    }
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventTest.java
    new file mode 100644
    index 0000000000..00bea9f601
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventTest.java
    @@ -0,0 +1,46 @@
    +/*
    + * 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.events;
    +
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +
    +import org.junit.jupiter.api.Test;
    +
    +class WorkflowOperationEventTest {
    +
    +    @Test
    +    void triggerEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.triggerEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.TRIGGER, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +
    +    @Test
    +    void pauseEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.pauseEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.PAUSE, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +
    +    @Test
    +    void killEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.killEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.KILL, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +}
    diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtilsTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtilsTest.java
    new file mode 100644
    index 0000000000..440f33b474
    --- /dev/null
    +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtilsTest.java
    @@ -0,0 +1,18 @@
    +package org.apache.dolphinscheduler.server.master.utils;
    +
    +import static org.junit.jupiter.api.Assertions.assertFalse;
    +import static org.junit.jupiter.api.Assertions.assertTrue;
    +
    +import org.junit.jupiter.api.Test;
    +import org.springframework.dao.DataAccessResourceFailureException;
    +
    +class ExceptionUtilsTest {
    +
    +    @Test
    +    void isDatabaseConnectedFailedException() {
    +        // todo: Directly connect to database
    +        assertTrue(ExceptionUtils.isDatabaseConnectedFailedException(
    +                new DataAccessResourceFailureException("Database connection failed")));
    +        assertFalse(ExceptionUtils.isDatabaseConnectedFailedException(new RuntimeException("runtime exception")));
    +    }
    +}
    diff --git a/dolphinscheduler-workflow-engine/README.md b/dolphinscheduler-workflow-engine/README.md
    new file mode 100644
    index 0000000000..6a14acf775
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/README.md
    @@ -0,0 +1,2 @@
    +# Introduction
    + 
    \ No newline at end of file
    diff --git a/dolphinscheduler-workflow-engine/pom.xml b/dolphinscheduler-workflow-engine/pom.xml
    new file mode 100644
    index 0000000000..40a360e7c8
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/pom.xml
    @@ -0,0 +1,36 @@
    +
    +
    +    4.0.0
    +    
    +        org.apache.dolphinscheduler
    +        dolphinscheduler
    +        dev-SNAPSHOT
    +    
    +
    +    dolphinscheduler-workflow-engine
    +
    +    
    +        
    +            
    +                org.apache.dolphinscheduler
    +                dolphinscheduler-bom
    +                ${project.version}
    +                pom
    +                import
    +            
    +        
    +    
    +
    +    
    +        
    +            org.apache.dolphinscheduler
    +            dolphinscheduler-common
    +            ${project.version}
    +        
    +        
    +            org.springframework
    +            spring-tx
    +        
    +    
    +
    diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAG.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAG.java
    new file mode 100644
    index 0000000000..63e6370887
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAG.java
    @@ -0,0 +1,146 @@
    +/*
    + * 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.workflow.engine.dag;
    +
    +import java.util.List;
    +import java.util.stream.Collectors;
    +
    +/**
    + * The Directed Acyclic Graph class.
    + * 

    + * The DAG is a directed graph, which contains the nodes and the edges, the nodeName is the unique identifier of the node. + * The nodes are the tasks, the edges are the dependencies between the tasks. + * The DAG is acyclic, which means there is no cycle in the graph. + * The DAG is a directed graph, which means the edges have direction. + * + */ +public interface DAG { + + /** + * Get the direct post node of given dagNode, if the dagNode is null, return the nodes which doesn't have inDegrees. + * e.g. The DAG is: + *

    +     *      {@code
    +     *          1 -> 2 -> 3
    +     *          4 -> 5
    +     *          6
    +     *      }
    +     * 
    + *
  • The post node of 1 is 2. + *
  • The post node of 3 is empty. + *
  • The post node of null is 1,4,6. + * + * @param dagNode the node of the DAG, can be null. + * @return post node list, sort by priority. + */ + List getDirectPostNodes(DAGNode dagNode); + + /** + * Same with {@link #getDirectPostNodes(DAGNode)}. + *

    + * If the dagNodeName is null, return the nodes which doesn't have inDegrees. Otherwise, return the post nodes of + * the given dagNodeName. If the dagNodeName is not null and cannot find the node in DAG, throw IllegalArgumentException. + * + * @param dagNodeName task name, can be null. + * @return post task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPostNodes(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPostNodes(dagNode); + } + + /** + * Same with {@link #getDirectPostNodes(String)}. Return the post node names. + * + * @param dagNodeName task name, can be null. + * @return post task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPostNodeNames(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPostNodes(dagNode).stream().map(DAGNode::getNodeName).collect(Collectors.toList()); + } + + /** + * Get the direct pre node of given dagNode, if the dagNode is null, return the nodes which doesn't have outDegrees. + * e.g. The DAG is: + *

    +     *      {@code
    +     *          1 -> 2 -> 3
    +     *          4 -> 5
    +     *          6
    +     *      }
    +     * 
    + *
  • The pre node of 1 is empty. + *
  • The pre node of 3 is 2. + *
  • The pre node of null is 3,5,6. + * + * @param dagNode the node of the DAG, can be null. + * @return pre node list, sort by priority. + */ + List getDirectPreNodes(DAGNode dagNode); + + /** + * Same with {@link #getDirectPreNodes(DAGNode)}. + *

    + * If the dagNodeName is null, return the nodes which doesn't have outDegrees. Otherwise, return the pre nodes of + * the given dagNodeName. If the dagNodeName is not null and cannot find the node in DAG, throw IllegalArgumentException. + * + * @param dagNodeName task name, can be null. + * @return pre task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPreNodes(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPreNodes(dagNode); + } + + /** + * Same with {@link #getDirectPreNodes(String)}. Return the pre node names. + * + * @param dagNodeName task name, can be null. + * @return pre task name list, sort by priority. + * @throws IllegalArgumentException if the dagNodeName is not null and cannot find the node in DAG. + */ + default List getDirectPreNodeNames(String dagNodeName) { + DAGNode dagNode = getDAGNode(dagNodeName); + if (dagNodeName != null && dagNode == null) { + throw new IllegalArgumentException("Cannot find the Node: " + dagNodeName + " in DAG"); + } + return getDirectPreNodes(dagNode).stream().map(DAGNode::getNodeName).collect(Collectors.toList()); + } + + /** + * Get the node of the DAG by the node name. + * + * @param nodeName the node name. + * @return the node of the DAG, return null if cannot find the node. + */ + DAGNode getDAGNode(String nodeName); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGEdge.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGEdge.java new file mode 100644 index 0000000000..8e2bdc2be6 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGEdge.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.dag; + +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * The edge of the DAG. + *

    + * The edge contains the fromNodeName and the toNodeName, the fromNodeName is the node name of the from node, the toNodeName is the node name of the to node. + *

    + * The formNodeName can be null, which means the edge is from the start node of the DAG. + * The toNodeName can be null, which means the edge is to the end node of the DAG. + * The fromNodeName and the toNodeName cannot be null at the same time. + */ +@Data +@Builder +@NoArgsConstructor +public class DAGEdge { + + private String fromNodeName; + private String toNodeName; + + public DAGEdge(String fromNodeName, String toNodeName) { + if (fromNodeName == null && toNodeName == null) { + throw new IllegalArgumentException("fromNodeName and toNodeName cannot be null at the same time" + + "fromNodeName: " + fromNodeName + ", toNodeName: " + toNodeName); + } + if (fromNodeName != null && fromNodeName.equals(toNodeName)) { + throw new IllegalArgumentException("fromNodeName and toNodeName cannot be the same" + + "fromNodeName: " + fromNodeName + ", toNodeName: " + toNodeName); + } + this.fromNodeName = fromNodeName; + this.toNodeName = toNodeName; + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNode.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNode.java new file mode 100644 index 0000000000..2e945a5463 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNode.java @@ -0,0 +1,92 @@ +/* + * 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.workflow.engine.dag; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; + +import lombok.Builder; +import lombok.Getter; +import lombok.NoArgsConstructor; + +/** + * The node of the DAG. + *

    + * The node contains the node name, the content of the node, the inDegrees and the outDegrees. + * The inDegrees is the edge from other nodes to the current node, the outDegrees is the edge from the current + * node to other nodes. + */ +@Getter +@Builder +@NoArgsConstructor +public class DAGNode implements IDAGNode { + + private String nodeName; + + /** + * whether the node is skipped, default is false, which means the node is not skipped. + * If the node is skipped, the node will not be executed. + */ + @Builder.Default + private boolean skip = false; + + private List inDegrees; + private List outDegrees; + + public DAGNode(String nodeName, + List inDegrees, + List outDegrees) { + this(nodeName, false, inDegrees, outDegrees); + } + + public DAGNode(String nodeName, + boolean skip, + List inDegrees, + List outDegrees) { + if (StringUtils.isEmpty(nodeName)) { + throw new IllegalArgumentException("nodeName cannot be empty"); + } + + if (CollectionUtils.isNotEmpty(inDegrees)) { + inDegrees.forEach(dagEdge -> { + if (!nodeName.equals(dagEdge.getToNodeName())) { + throw new IllegalArgumentException( + "The toNodeName of inDegree should be the nodeName of the node: " + + nodeName + ", inDegree: " + dagEdge); + } + }); + } + + if (CollectionUtils.isNotEmpty(outDegrees)) { + outDegrees.forEach(dagEdge -> { + if (!nodeName.equals(dagEdge.getFromNodeName())) { + throw new IllegalArgumentException( + "The fromNodeName of outDegree should be the nodeName of the node: " + + nodeName + ", outDegree: " + dagEdge); + } + }); + } + + this.nodeName = nodeName; + this.inDegrees = inDegrees; + this.outDegrees = outDegrees; + this.skip = skip; + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeDefinition.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeDefinition.java new file mode 100644 index 0000000000..b72d8ee6ba --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeDefinition.java @@ -0,0 +1,35 @@ +/* + * 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.workflow.engine.dag; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class DAGNodeDefinition { + + private String nodeName; + + private boolean skip; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNode.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNode.java new file mode 100644 index 0000000000..c3d85ae3ac --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNode.java @@ -0,0 +1,32 @@ +/* + * 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.workflow.engine.dag; + +import java.util.List; + +public interface IDAGNode { + + String getNodeName(); + + boolean isSkip(); + + List getInDegrees(); + + List getOutDegrees(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNodeAction.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNodeAction.java new file mode 100644 index 0000000000..9b6684a37f --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/IDAGNodeAction.java @@ -0,0 +1,35 @@ +/* + * 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.workflow.engine.dag; + +/** + * The IDAGNodeAction represent the action of a DAG node. + */ +public interface IDAGNodeAction { + + void run(); + + void kill(); + + void pause(); + + void success(); + + void failure(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAG.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAG.java new file mode 100644 index 0000000000..c31749af93 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAG.java @@ -0,0 +1,75 @@ +/* + * 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.workflow.engine.dag; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * The IWorkflowDAG represent the DAG of a workflow. + */ +public class WorkflowDAG implements DAG { + + private final Map dagNodeMap; + + public WorkflowDAG(List dagNodes) { + this.dagNodeMap = dagNodes.stream().collect(Collectors.toMap(DAGNode::getNodeName, Function.identity())); + } + + @Override + public List getDirectPostNodes(DAGNode dagNode) { + final String nodeName = dagNode.getNodeName(); + if (!dagNodeMap.containsKey(nodeName)) { + return Collections.emptyList(); + } + DAGNode node = dagNodeMap.get(nodeName); + List dagNodes = new ArrayList<>(); + for (DAGEdge edge : node.getOutDegrees()) { + if (dagNodeMap.containsKey(edge.getToNodeName())) { + dagNodes.add(dagNodeMap.get(edge.getToNodeName())); + } + } + return dagNodes; + } + + @Override + public List getDirectPreNodes(DAGNode dagNode) { + final String nodeName = dagNode.getNodeName(); + if (!dagNodeMap.containsKey(nodeName)) { + return Collections.emptyList(); + } + DAGNode node = dagNodeMap.get(nodeName); + List dagNodes = new ArrayList<>(); + for (DAGEdge edge : node.getInDegrees()) { + if (dagNodeMap.containsKey(edge.getFromNodeName())) { + dagNodes.add(dagNodeMap.get(edge.getFromNodeName())); + } + } + return dagNodes; + } + + @Override + public DAGNode getDAGNode(String nodeName) { + return dagNodeMap.get(nodeName); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilder.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilder.java new file mode 100644 index 0000000000..fbf666d947 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilder.java @@ -0,0 +1,107 @@ +/* + * 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.workflow.engine.dag; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Used to build WorkflowDAG, you need to add TaskNode first, then add TaskEdge. + * After adding all the TaskNodes and TaskEdges, you can call the build method to get the WorkflowDAG. + *

    + * Example: + *

    + *     {@code
    + *          WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    + *                 .addTaskNode(taskNodeA)
    + *                 .addTaskNode(taskNodeB)
    + *                 .addTaskNode(taskNodeC)
    + *                 .addTaskEdge(edgeAB)
    + *                 .addTaskEdge(edgeBC)
    + *                 .build();
    + *     }
    + * 
    + */ +public class WorkflowDAGBuilder { + + private final Map taskNameMap; + + private WorkflowDAGBuilder() { + this.taskNameMap = new HashMap<>(); + } + + public static WorkflowDAGBuilder newBuilder() { + return new WorkflowDAGBuilder(); + } + + public WorkflowDAGBuilder addTaskNodes(List dagNodes) { + dagNodes.forEach(this::addTaskNode); + return this; + } + + public WorkflowDAGBuilder addTaskNode(DAGNodeDefinition dagNodeDefinition) { + String nodeName = dagNodeDefinition.getNodeName(); + if (taskNameMap.containsKey(nodeName)) { + throw new IllegalArgumentException("TaskNode with name " + nodeName + " already exists"); + } + + DAGNode taskNode = DAGNode.builder() + .nodeName(nodeName) + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .skip(dagNodeDefinition.isSkip()) + .build(); + taskNameMap.put(nodeName, taskNode); + return this; + } + + public WorkflowDAGBuilder addTaskEdges(List processTaskRelations) { + processTaskRelations.forEach(this::addTaskEdge); + return this; + } + + public WorkflowDAGBuilder addTaskEdge(DAGEdge dagEdge) { + String fromNodeName = dagEdge.getFromNodeName(); + String toNodeName = dagEdge.getToNodeName(); + + if (taskNameMap.containsKey(fromNodeName)) { + DAGNode fromTask = taskNameMap.get(fromNodeName); + if (fromTask.getOutDegrees().contains(dagEdge)) { + throw new IllegalArgumentException( + "Edge from " + fromNodeName + " to " + toNodeName + " already exists"); + } + fromTask.getOutDegrees().add(dagEdge); + } + if (taskNameMap.containsKey(toNodeName)) { + DAGNode toTask = taskNameMap.get(toNodeName); + if (toTask.getInDegrees().contains(dagEdge)) { + throw new IllegalArgumentException( + "Edge from " + fromNodeName + " to " + toNodeName + " already exists"); + } + toTask.getInDegrees().add(dagEdge); + } + return this; + } + + public WorkflowDAG build() { + return new WorkflowDAG(new ArrayList<>(taskNameMap.values())); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngine.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngine.java new file mode 100644 index 0000000000..d5273367ef --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngine.java @@ -0,0 +1,171 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.dag.DAGNode; +import org.apache.dolphinscheduler.workflow.engine.event.TaskOperationEvent; +import org.apache.dolphinscheduler.workflow.engine.event.TaskOperationType; +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnableFactory; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionDAG; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DAGEngine implements IDAGEngine { + + private final IWorkflowExecutionContext workflowExecutionContext; + + private final ITaskExecutionRunnableFactory taskExecutionRunnableFactory; + + public DAGEngine(IWorkflowExecutionContext workflowExecutionContext, + ITaskExecutionRunnableFactory taskExecutionRunnableFactory) { + this.workflowExecutionContext = workflowExecutionContext; + this.taskExecutionRunnableFactory = taskExecutionRunnableFactory; + } + + @Override + public void triggerNextTasks(String parentTaskNodeName) { + workflowExecutionContext.getWorkflowExecutionDAG() + .getDirectPostNodeNames(parentTaskNodeName) + .forEach(this::triggerTask); + } + + @Override + public void triggerTask(String taskName) { + IWorkflowExecutionDAG workflowExecutionDAG = workflowExecutionContext.getWorkflowExecutionDAG(); + DAGNode dagNode = workflowExecutionDAG.getDAGNode(taskName); + if (dagNode == null) { + log.error("Cannot find the DAGNode for task: {}", taskName); + return; + } + + // todo: Use condition check? + // How to make sure the + if (!workflowExecutionDAG.isTaskAbleToBeTriggered(taskName)) { + log.info("The task: {} is not able to be triggered", taskName); + return; + } + + if (dagNode.isSkip()) { + log.info("The task: {} is skipped", taskName); + triggerNextTasks(taskName); + return; + } + + ITaskExecutionRunnable taskExecutionRunnable = + taskExecutionRunnableFactory.createTaskExecutionRunnable(taskName, workflowExecutionContext); + workflowExecutionDAG.storeTaskExecutionRunnable(taskExecutionRunnable); + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.RUN) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void failoverTask(Integer taskInstanceId) { + IWorkflowExecutionDAG workflowExecutionDAG = workflowExecutionContext.getWorkflowExecutionDAG(); + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionDAG.getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + ITaskExecutionRunnable failoverTaskExecutionRunnable = taskExecutionRunnableFactory + .createFailoverTaskExecutionRunnable(taskExecutionRunnable, workflowExecutionContext); + workflowExecutionDAG.storeTaskExecutionRunnable(failoverTaskExecutionRunnable); + + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.FAILOVER) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void retryTask(Integer taskInstanceId) { + IWorkflowExecutionDAG workflowExecutionDAG = workflowExecutionContext.getWorkflowExecutionDAG(); + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionDAG.getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + ITaskExecutionRunnable retryTaskExecutionRunnable = taskExecutionRunnableFactory + .createRetryTaskExecutionRunnable(taskExecutionRunnable, workflowExecutionContext); + workflowExecutionDAG.storeTaskExecutionRunnable(retryTaskExecutionRunnable); + + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.RETRY) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void pauseAllTask() { + workflowExecutionContext.getWorkflowExecutionDAG() + .getActiveTaskExecutionRunnable() + .stream() + .map(taskExecutionRunnable -> taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId()) + .forEach(this::pauseTask); + } + + @Override + public void pauseTask(Integer taskInstanceId) { + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.PAUSE) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + + @Override + public void killAllTask() { + workflowExecutionContext.getWorkflowExecutionDAG() + .getActiveTaskExecutionRunnable() + .stream() + .map(taskExecutionRunnable -> taskExecutionRunnable.getTaskExecutionContext().getTaskInstance().getId()) + .forEach(this::killTask); + } + + @Override + public void killTask(Integer taskInstanceId) { + ITaskExecutionRunnable taskExecutionRunnable = + workflowExecutionContext.getWorkflowExecutionDAG().getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + log.error("Cannot find the ITaskExecutionRunnable for taskInstance: {}", taskInstanceId); + return; + } + + TaskOperationEvent taskOperationEvent = TaskOperationEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .taskOperationType(TaskOperationType.KILL) + .build(); + workflowExecutionContext.getEventRepository().storeEventToTail(taskOperationEvent); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngineFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngineFactory.java new file mode 100644 index 0000000000..c07ca6ffbb --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/DAGEngineFactory.java @@ -0,0 +1,38 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnableFactory; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DAGEngineFactory implements IDAGEngineFactory { + + private final ITaskExecutionRunnableFactory taskExecutionRunnableFactory; + + public DAGEngineFactory(ITaskExecutionRunnableFactory taskExecutionRunnableFactory) { + this.taskExecutionRunnableFactory = taskExecutionRunnableFactory; + } + + @Override + public IDAGEngine createDAGEngine(IWorkflowExecutionContext workflowExecutionContext) { + return new DAGEngine(workflowExecutionContext, taskExecutionRunnableFactory); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngine.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngine.java new file mode 100644 index 0000000000..0971e2a179 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngine.java @@ -0,0 +1,72 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionDAG; + +/** + * The IDAGEngine is responsible for triggering, killing, pausing, and finalizing task in {@link org.apache.dolphinscheduler.workflow.engine.dag.WorkflowDAG}. + *

    All DAG operation should directly use the method in IDAGEngine, new {@link IWorkflowExecutionDAG} should be triggered by new IDAGEngine. + */ +public interface IDAGEngine { + + /** + * Trigger the tasks which are post of the given task. + *

    If there are no task after the given taskNode, will try to finish the WorkflowExecutionRunnable. + *

    If the + * + * @param parentTaskNodeName the parent task name + */ + void triggerNextTasks(String parentTaskNodeName); + + /** + * Trigger the given task + * + * @param taskName task name + */ + void triggerTask(String taskName); + + /** + * Failover the given task. + * + * @param taskInstanceId taskInstanceId + */ + void failoverTask(Integer taskInstanceId); + + /** + * Retry the given task. + * + * @param taskInstanceId taskInstanceId + */ + void retryTask(Integer taskInstanceId); + + void pauseAllTask(); + + /** + * Pause the given task. + */ + void pauseTask(Integer taskInstanceId); + + void killAllTask(); + + /** + * Kill the given task. + */ + void killTask(Integer taskId); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngineFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngineFactory.java new file mode 100644 index 0000000000..c0d134ad5c --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IDAGEngineFactory.java @@ -0,0 +1,26 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; + +public interface IDAGEngineFactory { + + IDAGEngine createDAGEngine(IWorkflowExecutionContext workflowExecutionContext); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngine.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngine.java new file mode 100644 index 0000000000..d1df4df7f5 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngine.java @@ -0,0 +1,58 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable; + +/** + * The WorkflowEngine is responsible for starting, stopping, pausing, and finalizing workflows. + */ +public interface IWorkflowEngine { + + /** + * 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. Once a workflow has been finalized, then it cannot receive new operation, and will be removed from memory. + * + * @param workflowInstanceId the ID of the workflow to finalize + */ + void finalizeWorkflow(Integer workflowInstanceId); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngineFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngineFactory.java new file mode 100644 index 0000000000..b011e0d16a --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/IWorkflowEngineFactory.java @@ -0,0 +1,24 @@ +/* + * 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.workflow.engine.engine; + +public interface IWorkflowEngineFactory { + + IWorkflowEngine createWorkflowEngine(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngine.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngine.java new file mode 100644 index 0000000000..92bec2fe1f --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngine.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.event.WorkflowOperationEvent; +import org.apache.dolphinscheduler.workflow.engine.exception.WorkflowExecuteRunnableNotFoundException; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class WorkflowEngine implements IWorkflowEngine { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + public WorkflowEngine(IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) { + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + } + + @Override + public void triggerWorkflow(IWorkflowExecutionRunnable workflowExecuteRunnable) { + IWorkflowExecutionContext workflowExecutionContext = workflowExecuteRunnable.getWorkflowExecutionContext(); + Integer workflowInstanceId = workflowExecutionContext.getWorkflowInstanceId(); + log.info("Triggering WorkflowExecutionRunnable: {}", workflowExecutionContext.getWorkflowInstanceName()); + workflowExecuteRunnableRepository.storeWorkflowExecutionRunnable(workflowExecuteRunnable); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.triggerEvent(workflowInstanceId)); + } + + @Override + public void pauseWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Pausing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.pauseEvent(workflowInstanceId)); + } + + @Override + public void killWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecuteRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecuteRunnable == null) { + throw new WorkflowExecuteRunnableNotFoundException(workflowInstanceId); + } + log.info("Killing WorkflowExecutionRunnable: {}", + workflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnable.storeEventToTail(WorkflowOperationEvent.killEvent(workflowInstanceId)); + } + + @Override + public void finalizeWorkflow(Integer workflowInstanceId) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(workflowInstanceId); + if (workflowExecutionRunnable == null) { + return; + } + log.info("Finalizing WorkflowExecutionRunnable: {}", + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstanceName()); + workflowExecuteRunnableRepository.removeWorkflowExecutionRunnable(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineFactory.java new file mode 100644 index 0000000000..5a32510bd1 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineFactory.java @@ -0,0 +1,28 @@ +/* + * 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.workflow.engine.engine; + +import org.apache.dolphinscheduler.workflow.engine.workflow.SingletonWorkflowExecuteRunnableRepository; + +public class WorkflowEngineFactory implements IWorkflowEngineFactory { + + @Override + public IWorkflowEngine createWorkflowEngine() { + return new WorkflowEngine(SingletonWorkflowExecuteRunnableRepository.getInstance()); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventEngine.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventEngine.java new file mode 100644 index 0000000000..35bfe079c5 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventEngine.java @@ -0,0 +1,111 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable; + +import org.apache.commons.lang3.time.StopWatch; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import org.slf4j.MDC; + +@Slf4j +public class EventEngine extends BaseDaemonThread { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + private final EventFirer eventFirer; + + private final Set firingWorkflowInstanceIds; + + public EventEngine(IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository, + EventFirer eventFirer) { + super("EventEngine"); + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + this.eventFirer = eventFirer; + this.firingWorkflowInstanceIds = ConcurrentHashMap.newKeySet(); + } + + @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.getActiveWorkflowExecutionRunnable(); + for (IWorkflowExecutionRunnable workflowExecutionRunnable : workflowExecutionRunnableCollection) { + IWorkflowExecutionContext workflowExecutionContext = + workflowExecutionRunnable.getWorkflowExecutionContext(); + final Integer workflowInstanceId = workflowExecutionContext.getWorkflowInstanceId(); + final String workflowInstanceName = workflowExecutionContext.getWorkflowInstanceName(); + try { + MDC.put(Constants.WORKFLOW_INSTANCE_ID_MDC_KEY, String.valueOf(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 { + if (fireCount > 0) { + log.info("Fire {} events for WorkflowExecutionRunnable: {} success", fireCount, + workflowInstanceName); + } + } + }); + } finally { + MDC.remove(Constants.WORKFLOW_INSTANCE_ID_MDC_KEY); + } + } + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventFirer.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventFirer.java new file mode 100644 index 0000000000..0026bd0c21 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventFirer.java @@ -0,0 +1,91 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.workflow.engine.utils.ExceptionUtils; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadPoolExecutor; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class EventFirer implements IEventFirer { + + private final IEventOperatorManager eventOperatorManager; + + private final ThreadPoolExecutor eventFireThreadPool; + + public EventFirer(IEventOperatorManager eventOperatorManager, int eventFireThreadPoolSize) { + this.eventOperatorManager = eventOperatorManager; + this.eventFireThreadPool = + ThreadUtils.newDaemonFixedThreadExecutor("EventFireThreadPool", eventFireThreadPoolSize); + } + + @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); + } + + 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).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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventOperatorManager.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventOperatorManager.java new file mode 100644 index 0000000000..fb0e3d0a50 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/EventOperatorManager.java @@ -0,0 +1,36 @@ +/* + * 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.workflow.engine.event; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +/** + * The event operator manager interface used to get {@link ITaskEventOperator}. + */ +@Slf4j +@Component +public class EventOperatorManager implements IEventOperatorManager { + + @Override + public IEventOperator getEventOperator(IEvent event) { + return null; + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IAsyncEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IAsyncEvent.java new file mode 100644 index 0000000000..2de1ebb9cc --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IAsyncEvent.java @@ -0,0 +1,24 @@ +/* + * 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.workflow.engine.event; + +/** + * 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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IDelayEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IDelayEvent.java new file mode 100644 index 0000000000..05a883d2ca --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IDelayEvent.java @@ -0,0 +1,26 @@ +/* + * 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.workflow.engine.event; + +public interface IDelayEvent { + + long getEventCreateTime(); + + long getDelayTime(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEvent.java new file mode 100644 index 0000000000..22fd7ec940 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEvent.java @@ -0,0 +1,22 @@ +/* + * 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.workflow.engine.event; + +public interface IEvent { + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventFirer.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventFirer.java new file mode 100644 index 0000000000..131bac9412 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventFirer.java @@ -0,0 +1,35 @@ +/* + * 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.workflow.engine.event; + +import java.util.concurrent.CompletableFuture; + +/** + * The event firer interface used to fire event. + * + */ +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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperator.java new file mode 100644 index 0000000000..be65031c06 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperator.java @@ -0,0 +1,32 @@ +/* + * 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.workflow.engine.event; + +/** + * 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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperatorManager.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperatorManager.java new file mode 100644 index 0000000000..68cffa4ecc --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventOperatorManager.java @@ -0,0 +1,33 @@ +/* + * 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.workflow.engine.event; + +/** + * 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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventRepository.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventRepository.java new file mode 100644 index 0000000000..f96df57680 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IEventRepository.java @@ -0,0 +1,37 @@ +/* + * 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.workflow.engine.event; + +import java.util.List; + +/** + * The event repository interface used to store event. + */ +public interface IEventRepository { + + void storeEventToTail(IEvent event); + + void storeEventToHead(IEvent event); + + IEvent poolEvent(); + + int getEventSize(); + + List getAllEvent(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ISyncEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ISyncEvent.java new file mode 100644 index 0000000000..5b4ecb6162 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ISyncEvent.java @@ -0,0 +1,21 @@ +/* + * 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.workflow.engine.event; + +public interface ISyncEvent { +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEvent.java new file mode 100644 index 0000000000..dd1f315aed --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEvent.java @@ -0,0 +1,26 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnable; + +public interface ITaskEvent extends IEvent { + + ITaskExecutionRunnable getTaskExecutionRunnable(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEventOperator.java new file mode 100644 index 0000000000..f015a61e2c --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/ITaskEventOperator.java @@ -0,0 +1,22 @@ +/* + * 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.workflow.engine.event; + +public interface ITaskEventOperator extends IEventOperator { + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEvent.java new file mode 100644 index 0000000000..304776b582 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEvent.java @@ -0,0 +1,29 @@ +/* + * 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.workflow.engine.event; + +public interface IWorkflowEvent extends IEvent { + + /** + * The id of WorkflowInstance which the event is related to + * + * @return workflowInstanceId, shouldn't be null + */ + Integer getWorkflowInstanceId(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEventOperator.java new file mode 100644 index 0000000000..48dc80ee65 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/IWorkflowEventOperator.java @@ -0,0 +1,23 @@ +/* + * 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.workflow.engine.event; + +public interface IWorkflowEventOperator + extends + IEventOperator { +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/MemoryEventRepository.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/MemoryEventRepository.java new file mode 100644 index 0000000000..f6f2c239fc --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/MemoryEventRepository.java @@ -0,0 +1,64 @@ +/* + * 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.workflow.engine.event; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class MemoryEventRepository implements IEventRepository { + + private final LinkedBlockingDeque eventQueue; + + private MemoryEventRepository() { + this.eventQueue = new LinkedBlockingDeque<>(); + } + + public static MemoryEventRepository newInstance() { + return new MemoryEventRepository(); + } + + @Override + public void storeEventToTail(IEvent event) { + log.info("Store event to tail: {}", event); + eventQueue.offerLast(event); + } + + @Override + public void storeEventToHead(IEvent event) { + eventQueue.offerFirst(event); + } + + @Override + public IEvent poolEvent() { + return eventQueue.poll(); + } + + @Override + public int getEventSize() { + return eventQueue.size(); + } + + public List getAllEvent() { + return new ArrayList<>(eventQueue); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEvent.java new file mode 100644 index 0000000000..1afb6ee7d8 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEvent.java @@ -0,0 +1,37 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnable; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class TaskOperationEvent implements ITaskEvent, ISyncEvent { + + private ITaskExecutionRunnable taskExecutionRunnable; + + private TaskOperationType taskOperationType; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEventOperator.java new file mode 100644 index 0000000000..e17868822c --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationEventOperator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.event; + +import org.apache.dolphinscheduler.workflow.engine.workflow.ITaskExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskOperationEventOperator implements ITaskEventOperator { + + @Override + public void handleEvent(TaskOperationEvent event) { + ITaskExecutionRunnable taskExecutionRunnable = event.getTaskExecutionRunnable(); + switch (event.getTaskOperationType()) { + case RUN: + taskExecutionRunnable.dispatch(); + break; + case KILL: + taskExecutionRunnable.kill(); + break; + case PAUSE: + taskExecutionRunnable.pause(); + break; + default: + log.error("Unknown TaskOperationType for event: {}", event); + } + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationType.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationType.java new file mode 100644 index 0000000000..094164c789 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/TaskOperationType.java @@ -0,0 +1,29 @@ +/* + * 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.workflow.engine.event; + +public enum TaskOperationType { + + FAILOVER, + RUN, + RETRY, + KILL, + PAUSE, + ; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFailedEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFailedEvent.java new file mode 100644 index 0000000000..b807e726fa --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFailedEvent.java @@ -0,0 +1,35 @@ +/* + * 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.workflow.engine.event; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowFailedEvent implements IWorkflowEvent { + + private Integer workflowInstanceId; + + private String failedReason; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEvent.java new file mode 100644 index 0000000000..0c23e15f2b --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEvent.java @@ -0,0 +1,32 @@ +/* + * 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.workflow.engine.event; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@AllArgsConstructor +@NoArgsConstructor +public class WorkflowFinalizeEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEventOperator.java new file mode 100644 index 0000000000..4c64e4c62d --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinalizeEventOperator.java @@ -0,0 +1,41 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecuteRunnableRepository; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class WorkflowFinalizeEventOperator + implements + IWorkflowEventOperator { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + public WorkflowFinalizeEventOperator(IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) { + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + } + + @Override + public void handleEvent(WorkflowFinalizeEvent event) { + Integer workflowInstanceId = event.getWorkflowInstanceId(); + workflowExecuteRunnableRepository.removeWorkflowExecutionRunnable(workflowInstanceId); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinishEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinishEvent.java new file mode 100644 index 0000000000..bd8a610599 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowFinishEvent.java @@ -0,0 +1,37 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class WorkflowFinishEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + + private WorkflowExecutionStatus workflowExecutionStatus; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEvent.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEvent.java new file mode 100644 index 0000000000..4c7e30808f --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEvent.java @@ -0,0 +1,51 @@ +/* + * 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.workflow.engine.event; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +@AllArgsConstructor +public class WorkflowOperationEvent implements IWorkflowEvent, ISyncEvent { + + private Integer workflowInstanceId; + private WorkflowOperationType workflowOperationType; + + public static WorkflowOperationEvent of(Integer workflowInstanceId, WorkflowOperationType workflowOperationType) { + return WorkflowOperationEvent.builder() + .workflowInstanceId(workflowInstanceId) + .workflowOperationType(workflowOperationType) + .build(); + } + + public static WorkflowOperationEvent triggerEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.TRIGGER); + } + + public static WorkflowOperationEvent pauseEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.PAUSE); + } + + public static WorkflowOperationEvent killEvent(Integer workflowInstanceId) { + return of(workflowInstanceId, WorkflowOperationType.KILL); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventOperator.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventOperator.java new file mode 100644 index 0000000000..59a1d62e05 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventOperator.java @@ -0,0 +1,84 @@ +/* + * 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.workflow.engine.event; + +import org.apache.dolphinscheduler.workflow.engine.utils.ExceptionUtils; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecuteRunnableRepository; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionContext; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class WorkflowOperationEventOperator implements IWorkflowEventOperator { + + private final IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository; + + public WorkflowOperationEventOperator(IWorkflowExecuteRunnableRepository workflowExecuteRunnableRepository) { + this.workflowExecuteRunnableRepository = workflowExecuteRunnableRepository; + } + + @Override + public void handleEvent(WorkflowOperationEvent event) { + IWorkflowExecutionRunnable workflowExecutionRunnable = + workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(event.getWorkflowInstanceId()); + if (workflowExecutionRunnable == null) { + log.warn("WorkflowExecutionRunnable not found: {}", event); + return; + } + switch (event.getWorkflowOperationType()) { + case TRIGGER: + triggerWorkflow(workflowExecutionRunnable); + break; + case PAUSE: + pauseWorkflow(workflowExecutionRunnable); + break; + case KILL: + killWorkflow(workflowExecutionRunnable); + break; + default: + log.error("Unknown operationType for event: {}", event); + } + } + + private void triggerWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + try { + workflowExecutionRunnable.start(); + } catch (Throwable exception) { + if (ExceptionUtils.isDatabaseConnectedFailedException(exception)) { + throw exception; + } + IWorkflowExecutionContext workflowExecutionContext = + workflowExecutionRunnable.getWorkflowExecutionContext(); + log.error("Trigger workflow: {} failed", workflowExecutionContext.getWorkflowInstanceName(), exception); + WorkflowFailedEvent workflowExecutionRunnableFailedEvent = WorkflowFailedEvent.builder() + .workflowInstanceId(workflowExecutionContext.getWorkflowInstanceId()) + .failedReason(exception.getMessage()) + .build(); + workflowExecutionRunnable.storeEventToTail(workflowExecutionRunnableFailedEvent); + } + } + + private void pauseWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnable.pause(); + } + + private void killWorkflow(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnable.kill(); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationType.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationType.java new file mode 100644 index 0000000000..0a1d260217 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationType.java @@ -0,0 +1,36 @@ +/* + * 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.workflow.engine.event; + +public enum WorkflowOperationType { + + /** + * Trigger the workflow instance. + */ + TRIGGER, + /** + * Pause the workflow instance, it will pause the running task instances. + */ + PAUSE, + /** + * Kill the workflow instance, it will kill the running task instances. + */ + KILL, + ; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/exception/WorkflowExecuteRunnableNotFoundException.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/exception/WorkflowExecuteRunnableNotFoundException.java new file mode 100644 index 0000000000..10d0b4252b --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/exception/WorkflowExecuteRunnableNotFoundException.java @@ -0,0 +1,30 @@ +/* + * 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.workflow.engine.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-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/utils/ExceptionUtils.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/utils/ExceptionUtils.java new file mode 100644 index 0000000000..625ff86e8d --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/utils/ExceptionUtils.java @@ -0,0 +1,28 @@ +/* + * 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.workflow.engine.utils; + +import org.springframework.dao.DataAccessResourceFailureException; + +public class ExceptionUtils { + + public static boolean isDatabaseConnectedFailedException(Throwable e) { + return e instanceof DataAccessResourceFailureException; + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IEventfulExecutionRunnable.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IEventfulExecutionRunnable.java new file mode 100644 index 0000000000..08bfa53055 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IEventfulExecutionRunnable.java @@ -0,0 +1,34 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.event.IEvent; +import org.apache.dolphinscheduler.workflow.engine.event.IEventRepository; + +public interface IEventfulExecutionRunnable { + + IEventRepository getEventRepository(); + + default void storeEventToTail(IEvent event) { + getEventRepository().storeEventToTail(event); + } + + default void storeEventToHead(IEvent event) { + getEventRepository().storeEventToHead(event); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContext.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContext.java new file mode 100644 index 0000000000..6a2945244c --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContext.java @@ -0,0 +1,24 @@ +/* + * 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.workflow.engine.workflow; + +public interface ITaskExecutionContext { + + ITaskInstance getTaskInstance(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContextFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContextFactory.java new file mode 100644 index 0000000000..2f6a0bd9b8 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionContextFactory.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.workflow; + +public interface ITaskExecutionContextFactory { + + ITaskExecutionContext createTaskExecutionContext(String taskName, + IWorkflowExecutionContext workflowExecutionContext); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnable.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnable.java new file mode 100644 index 0000000000..a3ab6cc6a6 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnable.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.workflow; + +/** + * The TaskExecutionRunnable represent the running task, it is responsible for operate the task instance. e.g. dispatch, kill, pause. + */ +public interface ITaskExecutionRunnable { + + /** + * Dispatch the task instance. + */ + void dispatch(); + + /** + * Run the task instance. + */ + void run(); + + /** + * Kill the task instance. + */ + void kill(); + + /** + * Pause the task instance. + */ + void pause(); + + /** + * Get the task execution context. + * + * @return the task execution context + */ + ITaskExecutionContext getTaskExecutionContext(); + + /** + * Determine whether the current task is ready to trigger the post task node. + * + * @param taskNodeName post task name + * @return true if the current task can be accessed to the post task. + */ + boolean isReadyToTrigger(String taskNodeName); +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableFactory.java new file mode 100644 index 0000000000..b23364597b --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableFactory.java @@ -0,0 +1,30 @@ +/* + * 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.workflow.engine.workflow; + +public interface ITaskExecutionRunnableFactory { + + ITaskExecutionRunnable createTaskExecutionRunnable(String taskName, + IWorkflowExecutionContext workflowExecutionContext); + + ITaskExecutionRunnable createFailoverTaskExecutionRunnable(ITaskExecutionRunnable taskExecutionRunnable, + IWorkflowExecutionContext workflowExecutionContext); + + ITaskExecutionRunnable createRetryTaskExecutionRunnable(ITaskExecutionRunnable taskExecutionRunnable, + IWorkflowExecutionContext workflowExecutionContext); +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableRepository.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableRepository.java new file mode 100644 index 0000000000..d78f305665 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskExecutionRunnableRepository.java @@ -0,0 +1,34 @@ +/* + * 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.workflow.engine.workflow; + +import java.util.Collection; + +public interface ITaskExecutionRunnableRepository { + + void storeTaskExecutionRunnable(ITaskExecutionRunnable taskExecutionRunnable); + + ITaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId); + + ITaskExecutionRunnable getTaskExecutionRunnableByName(String taskInstanceName); + + Collection getActiveTaskExecutionRunnable(); + + void removeTaskExecutionRunnable(Integer taskInstanceId); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskInstance.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskInstance.java new file mode 100644 index 0000000000..2e86662267 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/ITaskInstance.java @@ -0,0 +1,26 @@ +/* + * 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.workflow.engine.workflow; + +public interface ITaskInstance { + + int getId(); + + String getName(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecuteRunnableRepository.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecuteRunnableRepository.java new file mode 100644 index 0000000000..758cd99a27 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecuteRunnableRepository.java @@ -0,0 +1,33 @@ +/* + * 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.workflow.engine.workflow; + +import java.util.Collection; + +public interface IWorkflowExecuteRunnableRepository { + + void storeWorkflowExecutionRunnable(IWorkflowExecutionRunnable workflowExecutionRunnable); + + IWorkflowExecutionRunnable getWorkflowExecutionRunnableById(Integer workflowInstanceId); + + Collection getActiveWorkflowExecutionRunnable(); + + void removeWorkflowExecutionRunnable(Integer workflowInstanceId); + + void clear(); +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionContext.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionContext.java new file mode 100644 index 0000000000..265076b4f1 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionContext.java @@ -0,0 +1,38 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.event.IEventRepository; + +public interface IWorkflowExecutionContext { + + IWorkflowInstance getWorkflowInstance(); + + IWorkflowExecutionDAG getWorkflowExecutionDAG(); + + IEventRepository getEventRepository(); + + default int getWorkflowInstanceId() { + return getWorkflowInstance().getId(); + } + + default String getWorkflowInstanceName() { + return getWorkflowInstance().getName(); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAG.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAG.java new file mode 100644 index 0000000000..d76ee1a64a --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAG.java @@ -0,0 +1,71 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.dag.DAG; + +import java.util.List; + +/** + * The WorkflowExecutionDAG represent the running workflow DAG. + */ +public interface IWorkflowExecutionDAG extends DAG { + + List getStartNodeNames(); + + /** + * Get TaskExecutionRunnable by given TaskInstanceId. + * + * @param taskInstanceId taskInstanceId. + * @return TaskExecutionRunnable + */ + ITaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId); + + /** + * Get TaskExecutionRunnable by given taskName. + * + * @param taskName task name. + * @return TaskExecutionRunnable + */ + ITaskExecutionRunnable getTaskExecutionRunnableByName(String taskName); + + /** + * Get TaskExecutionRunnable which is not finished. + * + * @return TaskExecutionRunnable + */ + List getActiveTaskExecutionRunnable(); + + /** + * Get the direct pre TaskExecutionRunnable of the given taskName. + * + * @param taskName task name. + * @return TaskExecutionRunnable + */ + List getDirectPreTaskExecutionRunnable(String taskName); + + /** + * Check whether the taskNode is ready to run. + * + * @param taskName taskNodeName + * @return true if the taskNode is ready to run. + */ + boolean isTaskAbleToBeTriggered(String taskName); + + void storeTaskExecutionRunnable(ITaskExecutionRunnable taskExecutionRunnable); +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAGFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAGFactory.java new file mode 100644 index 0000000000..dbac2888f4 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionDAGFactory.java @@ -0,0 +1,24 @@ +/* + * 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.workflow.engine.workflow; + +public interface IWorkflowExecutionDAGFactory { + + IWorkflowExecutionDAG createWorkflowExecutionDAG(IWorkflowExecutionContext workflowExecutionContext); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnable.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnable.java new file mode 100644 index 0000000000..1c6e75ba1e --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnable.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.engine.IDAGEngine; + +/** + * The IWorkflowExecuteRunnable represent a running workflow instance, it is responsible for operate the workflow instance. e.g. start, kill, pause. + */ +public interface IWorkflowExecutionRunnable extends IEventfulExecutionRunnable { + + /** + * Start the workflow instance. + */ + void start(); + + /** + * Kill the workflow instance. + */ + void kill(); + + /** + * Pause the workflow instance. + */ + void pause(); + + /** + * Get the workflow execution context. + * + * @return the workflow execution context + */ + IWorkflowExecutionContext getWorkflowExecutionContext(); + + /** + * Get the {@link IDAGEngine} which used to execute the dag of the workflow instance. + * + * @return dag engine. + */ + IDAGEngine getDagEngine(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnableFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnableFactory.java new file mode 100644 index 0000000000..f2c5042e4f --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowExecutionRunnableFactory.java @@ -0,0 +1,24 @@ +/* + * 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.workflow.engine.workflow; + +public interface IWorkflowExecutionRunnableFactory { + + IWorkflowExecutionRunnable createWorkflowExecutionRunnable(IWorkflowExecutionContext workflowExecutionContext); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowInstance.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowInstance.java new file mode 100644 index 0000000000..49944d248a --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/IWorkflowInstance.java @@ -0,0 +1,26 @@ +/* + * 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.workflow.engine.workflow; + +public interface IWorkflowInstance { + + int getId(); + + String getName(); + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/SingletonWorkflowExecuteRunnableRepository.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/SingletonWorkflowExecuteRunnableRepository.java new file mode 100644 index 0000000000..147f0076ba --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/SingletonWorkflowExecuteRunnableRepository.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.workflow.engine.workflow; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class SingletonWorkflowExecuteRunnableRepository implements IWorkflowExecuteRunnableRepository { + + private static final IWorkflowExecuteRunnableRepository INSTANCE = new SingletonWorkflowExecuteRunnableRepository(); + + private final Map workflowExecutionRunnableMap; + + private SingletonWorkflowExecuteRunnableRepository() { + this.workflowExecutionRunnableMap = new ConcurrentHashMap<>(); + } + + public static IWorkflowExecuteRunnableRepository getInstance() { + return INSTANCE; + } + + public void storeWorkflowExecutionRunnable(IWorkflowExecutionRunnable workflowExecutionRunnable) { + workflowExecutionRunnableMap.put( + workflowExecutionRunnable.getWorkflowExecutionContext().getWorkflowInstanceId(), + workflowExecutionRunnable); + } + + public IWorkflowExecutionRunnable getWorkflowExecutionRunnableById(Integer workflowInstanceId) { + return workflowExecutionRunnableMap.get(workflowInstanceId); + } + + public Collection getActiveWorkflowExecutionRunnable() { + return workflowExecutionRunnableMap.values(); + } + + public void removeWorkflowExecutionRunnable(Integer workflowInstanceId) { + workflowExecutionRunnableMap.remove(workflowInstanceId); + } + + @Override + public void clear() { + workflowExecutionRunnableMap.clear(); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContext.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContext.java new file mode 100644 index 0000000000..380ed8b1ee --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContext.java @@ -0,0 +1,32 @@ +/* + * 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.workflow.engine.workflow; + +public class TaskExecutionContext implements ITaskExecutionContext { + + private final ITaskInstance taskInstance; + + public TaskExecutionContext(ITaskInstance taskInstance) { + this.taskInstance = taskInstance; + } + + @Override + public ITaskInstance getTaskInstance() { + return taskInstance; + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContextFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContextFactory.java new file mode 100644 index 0000000000..446bb2f661 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionContextFactory.java @@ -0,0 +1,27 @@ +/* + * 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.workflow.engine.workflow; + +public class TaskExecutionContextFactory implements ITaskExecutionContextFactory { + + @Override + public ITaskExecutionContext createTaskExecutionContext(String taskName, + IWorkflowExecutionContext workflowExecutionContext) { + return null; + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnable.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnable.java new file mode 100644 index 0000000000..322cd8ee8d --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnable.java @@ -0,0 +1,57 @@ +/* + * 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.workflow.engine.workflow; + +public class TaskExecutionRunnable implements ITaskExecutionRunnable { + + public final ITaskExecutionContext taskExecutionContext; + + public TaskExecutionRunnable(ITaskExecutionContext taskExecutionContext) { + this.taskExecutionContext = taskExecutionContext; + } + + @Override + public void dispatch() { + + } + + @Override + public void run() { + + } + + @Override + public void kill() { + + } + + @Override + public void pause() { + + } + + @Override + public ITaskExecutionContext getTaskExecutionContext() { + return taskExecutionContext; + } + + @Override + public boolean isReadyToTrigger(String taskNodeName) { + return false; + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnableFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnableFactory.java new file mode 100644 index 0000000000..3e7b5ec7c2 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/TaskExecutionRunnableFactory.java @@ -0,0 +1,35 @@ +/* + * 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.workflow.engine.workflow; + +public class TaskExecutionRunnableFactory implements ITaskExecutionRunnableFactory { + + private ITaskExecutionContextFactory taskExecutionContextFactory; + + public TaskExecutionRunnableFactory(ITaskExecutionContextFactory taskExecutionContextFactory) { + this.taskExecutionContextFactory = taskExecutionContextFactory; + } + + @Override + public ITaskExecutionRunnable createTaskExecutionRunnable(String taskName, + IWorkflowExecutionContext workflowExecutionContext) { + ITaskExecutionContext taskExecutionContext = + taskExecutionContextFactory.createTaskExecutionContext(taskName, workflowExecutionContext); + return new TaskExecutionRunnable(taskExecutionContext); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionContext.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionContext.java new file mode 100644 index 0000000000..6cca1eced1 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionContext.java @@ -0,0 +1,39 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.event.IEventRepository; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@AllArgsConstructor +@NoArgsConstructor +public class WorkflowExecutionContext implements IWorkflowExecutionContext { + + private IWorkflowInstance workflowInstance; + + private IWorkflowExecutionDAG workflowExecutionDAG; + + private IEventRepository eventRepository; + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionDAG.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionDAG.java new file mode 100644 index 0000000000..01fafcc6b7 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionDAG.java @@ -0,0 +1,124 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.dag.DAGNode; +import org.apache.dolphinscheduler.workflow.engine.dag.WorkflowDAG; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +/** + * The WorkflowExecutionDAG represent a running workflow instance DAG. + */ +@Slf4j +public class WorkflowExecutionDAG implements IWorkflowExecutionDAG { + + private final ITaskExecutionRunnableRepository taskExecutionRunnableRepository; + + private final WorkflowDAG workflowDAG; + + @Getter + private final Set startNodeNames; + + public WorkflowExecutionDAG(ITaskExecutionRunnableRepository taskExecutionRunnableRepository, + WorkflowDAG workflowDAG) { + this(taskExecutionRunnableRepository, workflowDAG, Collections.emptySet()); + } + + public WorkflowExecutionDAG(ITaskExecutionRunnableRepository taskExecutionRunnableRepository, + WorkflowDAG workflowDAG, + Set startNodeNames) { + this.taskExecutionRunnableRepository = taskExecutionRunnableRepository; + this.workflowDAG = workflowDAG; + this.startNodeNames = startNodeNames; + } + + @Override + public ITaskExecutionRunnable getTaskExecutionRunnableById(Integer taskInstanceId) { + return taskExecutionRunnableRepository.getTaskExecutionRunnableById(taskInstanceId); + } + + @Override + public ITaskExecutionRunnable getTaskExecutionRunnableByName(String taskName) { + return taskExecutionRunnableRepository.getTaskExecutionRunnableByName(taskName); + } + + @Override + public List getActiveTaskExecutionRunnable() { + return new ArrayList<>(taskExecutionRunnableRepository.getActiveTaskExecutionRunnable()); + } + + @Override + public List getDirectPreTaskExecutionRunnable(String taskName) { + return getDirectPreNodeNames(taskName) + .stream() + .map(taskExecutionRunnableRepository::getTaskExecutionRunnableByName) + .collect(Collectors.toList()); + } + + @Override + public boolean isTaskAbleToBeTriggered(String taskNodeName) { + // todo: Check whether the workflow instance is finished or ready to finish. + List directPreNodes = getDirectPreNodes(taskNodeName); + if (log.isDebugEnabled()) { + log.debug("Begin to check whether the task {} is able to be triggered.", taskNodeName); + log.debug("Task {} directly dependent on the task: {}.", taskNodeName, + directPreNodes.stream().map(DAGNode::getNodeName).collect(Collectors.toList())); + } + for (DAGNode directPreNode : directPreNodes) { + if (directPreNode.isSkip()) { + log.debug("The task {} is skipped.", directPreNode.getNodeName()); + continue; + } + ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnableByName(directPreNode.getNodeName()); + if (taskExecutionRunnable == null || taskExecutionRunnable.isReadyToTrigger(taskNodeName)) { + log.debug("The task {} is not finished or not able to access to the task {}.", + directPreNode.getNodeName(), taskNodeName); + } + } + return true; + } + + @Override + public void storeTaskExecutionRunnable(ITaskExecutionRunnable taskExecutionRunnable) { + taskExecutionRunnableRepository.storeTaskExecutionRunnable(taskExecutionRunnable); + } + + @Override + public List getDirectPostNodes(DAGNode dagNode) { + return workflowDAG.getDirectPostNodes(dagNode); + } + + @Override + public List getDirectPreNodes(DAGNode dagNode) { + return workflowDAG.getDirectPreNodes(dagNode); + } + + @Override + public DAGNode getDAGNode(String nodeName) { + return workflowDAG.getDAGNode(nodeName); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnable.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnable.java new file mode 100644 index 0000000000..7b05d22048 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnable.java @@ -0,0 +1,64 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.engine.IDAGEngine; +import org.apache.dolphinscheduler.workflow.engine.event.IEventRepository; + +import org.apache.commons.collections4.CollectionUtils; + +import java.util.List; + +import lombok.Getter; + +@Getter +public class WorkflowExecutionRunnable implements IWorkflowExecutionRunnable { + + private final IWorkflowExecutionContext workflowExecutionContext; + + private final IDAGEngine dagEngine; + + public WorkflowExecutionRunnable(IWorkflowExecutionContext workflowExecutionContext, IDAGEngine dagEngine) { + this.workflowExecutionContext = workflowExecutionContext; + this.dagEngine = dagEngine; + } + + public void start() { + List workflowStartNodeNames = workflowExecutionContext.getWorkflowExecutionDAG().getStartNodeNames(); + if (CollectionUtils.isEmpty(workflowStartNodeNames)) { + dagEngine.triggerNextTasks(null); + } else { + workflowStartNodeNames.forEach(dagEngine::triggerTask); + } + } + + @Override + public void pause() { + dagEngine.pauseAllTask(); + } + + @Override + public void kill() { + dagEngine.killAllTask(); + } + + @Override + public IEventRepository getEventRepository() { + return workflowExecutionContext.getEventRepository(); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnableFactory.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnableFactory.java new file mode 100644 index 0000000000..ebfb3510db --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowExecutionRunnableFactory.java @@ -0,0 +1,36 @@ +/* + * 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.workflow.engine.workflow; + +import org.apache.dolphinscheduler.workflow.engine.engine.IDAGEngine; +import org.apache.dolphinscheduler.workflow.engine.engine.IDAGEngineFactory; + +public class WorkflowExecutionRunnableFactory implements IWorkflowExecutionRunnableFactory { + + private final IDAGEngineFactory dagEngineFactory; + + public WorkflowExecutionRunnableFactory(IDAGEngineFactory dagEngineFactory) { + this.dagEngineFactory = dagEngineFactory; + } + + @Override + public WorkflowExecutionRunnable createWorkflowExecutionRunnable(IWorkflowExecutionContext workflowExecutionContext) { + IDAGEngine dagEngine = dagEngineFactory.createDAGEngine(workflowExecutionContext); + return new WorkflowExecutionRunnable(workflowExecutionContext, dagEngine); + } +} diff --git a/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowInstance.java b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowInstance.java new file mode 100644 index 0000000000..5280e86c08 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/main/java/org/apache/dolphinscheduler/workflow/engine/workflow/WorkflowInstance.java @@ -0,0 +1,38 @@ +/* + * 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.workflow.engine.workflow; + +import lombok.Getter; + +@Getter +public class WorkflowInstance implements IWorkflowInstance { + + private final int id; + + private final String name; + + public WorkflowInstance(int id, String name) { + this.id = id; + this.name = name; + } + + public static WorkflowInstance of(int id, String name) { + return new WorkflowInstance(id, name); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/SingletonWorkflowExecuteRunnableRepositoryAssertions.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/SingletonWorkflowExecuteRunnableRepositoryAssertions.java new file mode 100644 index 0000000000..1d4553d2a7 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/SingletonWorkflowExecuteRunnableRepositoryAssertions.java @@ -0,0 +1,37 @@ +/* + * 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.workflow.engine.assertions; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +import org.apache.dolphinscheduler.workflow.engine.workflow.SingletonWorkflowExecuteRunnableRepository; + +public class SingletonWorkflowExecuteRunnableRepositoryAssertions { + + public static void existWorkflowExecutionRunnable(Integer workflowInstanceId) { + assertNotNull(SingletonWorkflowExecuteRunnableRepository.getInstance() + .getWorkflowExecutionRunnableById(workflowInstanceId)); + } + + public static void notExistWorkflowExecutionRunnable(Integer workflowInstanceId) { + assertNull(SingletonWorkflowExecuteRunnableRepository.getInstance() + .getWorkflowExecutionRunnableById(workflowInstanceId)); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowDAGAssertion.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowDAGAssertion.java new file mode 100644 index 0000000000..3ccbc4702a --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowDAGAssertion.java @@ -0,0 +1,173 @@ +/* + * 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.workflow.engine.assertions; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.apache.dolphinscheduler.workflow.engine.dag.WorkflowDAG; + +import org.apache.commons.collections4.CollectionUtils; + +import java.util.ArrayList; +import java.util.List; + +public class WorkflowDAGAssertion { + + private final WorkflowDAG workflowDAG; + + private final List nodeAssertions; + + private WorkflowDAGAssertion(WorkflowDAG workflowDAG) { + this.workflowDAG = workflowDAG; + this.nodeAssertions = new ArrayList<>(); + } + + public static WorkflowDAGAssertion workflowDag(WorkflowDAG workflowDAG) { + return new WorkflowDAGAssertion(workflowDAG); + } + + public WorkflowDAGAssertion nodeAssertion(NodeAssertion nodeAssertion) { + nodeAssertions.add(nodeAssertion); + return this; + } + + public void doAssertion() { + nodeAssertions.forEach(nodeAssertion -> nodeAssertion.doAssertion(workflowDAG)); + } + + public static class NodeAssertion { + + /** + * node name of the assertion + */ + private final String nodeName; + + /** + * whether the node exist + */ + private boolean exist; + + /** + * whether the node is skipped + */ + private boolean skip; + + /** + * whether the node has out degree + */ + private List postNodes; + + /** + * whether the node has in degree + */ + private List preNodes; + + private NodeAssertion(String nodeName) { + this.nodeName = nodeName; + this.postNodes = new ArrayList<>(); + this.preNodes = new ArrayList<>(); + } + + public static NodeAssertion node(String nodeName) { + return new NodeAssertion(nodeName); + } + + public NodeAssertion exist() { + this.exist = true; + return this; + } + + public NodeAssertion skip() { + this.skip = true; + return this; + } + + public NodeAssertion noEdge() { + this.postNodes = new ArrayList<>(); + this.preNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion noInDegree() { + this.preNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion inDegrees(List preNodes) { + this.preNodes.addAll(preNodes); + return this; + } + + public NodeAssertion inDegrees(String preNode) { + this.preNodes.add(preNode); + return this; + } + + public NodeAssertion noOutDegree() { + this.postNodes = new ArrayList<>(); + return this; + } + + public NodeAssertion outDegrees(List postNodes) { + this.postNodes.addAll(postNodes); + return this; + } + + public NodeAssertion outDegrees(String postNode) { + this.postNodes.add(postNode); + return this; + } + + private void doAssertion(WorkflowDAG workflowDAG) { + if (exist) { + // node exist + assertNotNull(workflowDAG.getDAGNode(nodeName), "node " + nodeName + " does not exist"); + } else { + + assertNull(workflowDAG.getDAGNode(nodeName), "node " + nodeName + " exist"); + } + + if (skip) { + assertTrue(workflowDAG.getDAGNode(nodeName).isSkip(), "node " + nodeName + " is not skipped"); + } else { + assertFalse(workflowDAG.getDAGNode(nodeName).isSkip(), "node " + nodeName + " is skipped"); + } + + if (CollectionUtils.isEmpty(postNodes)) { + assertTrue(workflowDAG.getDirectPostNodeNames(nodeName).isEmpty(), + "node " + nodeName + " has outDegree " + workflowDAG.getDirectPostNodes(nodeName)); + } else { + postNodes + .forEach(postNode -> assertTrue(workflowDAG.getDirectPostNodeNames(nodeName).contains(postNode), + "node " + nodeName + " does has outDegree " + postNode)); + } + + if (CollectionUtils.isEmpty(preNodes)) { + assertTrue(workflowDAG.getDirectPreNodeNames(nodeName).isEmpty(), + "node " + nodeName + " has inDegree " + workflowDAG.getDirectPreNodeNames(nodeName)); + } else { + preNodes.forEach(preNode -> assertTrue(workflowDAG.getDirectPreNodeNames(nodeName).contains(preNode), + "node " + nodeName + " does has inDegree " + preNode)); + } + } + } + +} diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowExecutionRunnableAssertions.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowExecutionRunnableAssertions.java new file mode 100644 index 0000000000..494e66bcd6 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/assertions/WorkflowExecutionRunnableAssertions.java @@ -0,0 +1,49 @@ +/* + * 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.workflow.engine.assertions; + +import org.apache.dolphinscheduler.workflow.engine.event.IEvent; +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable; + +import org.opentest4j.AssertionFailedError; + +public class WorkflowExecutionRunnableAssertions { + + private final IWorkflowExecutionRunnable workflowExecutionRunnable; + + private WorkflowExecutionRunnableAssertions(IWorkflowExecutionRunnable workflowExecutionRunnable) { + this.workflowExecutionRunnable = workflowExecutionRunnable; + } + + public static WorkflowExecutionRunnableAssertions workflowExecutionRunnable(IWorkflowExecutionRunnable workflowExecutionRunnable) { + return new WorkflowExecutionRunnableAssertions(workflowExecutionRunnable); + } + + public void existEvent(IEvent event) { + if (event == null) { + throw new IllegalArgumentException("Event cannot be null"); + } + boolean exist = workflowExecutionRunnable.getEventRepository().getAllEvent() + .stream() + .anyMatch(event1 -> event1.equals(event1)); + if (!exist) { + throw new AssertionFailedError("The workflowExecuteRunnable doesn't exist event: " + event); + } + } + +} diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeTest.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeTest.java new file mode 100644 index 0000000000..0502450f82 --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/DAGNodeTest.java @@ -0,0 +1,104 @@ +package org.apache.dolphinscheduler.workflow.engine.dag; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.ArrayList; + +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +class DAGNodeTest { + + @Test + void buildDAGNode_EmptyNodeName() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build()); + assertEquals("nodeName cannot be empty", illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_BadInDegree() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .nodeName("A") + .inDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName(null) + .toNodeName("B") + .build())) + .outDegrees(new ArrayList<>()) + .build()); + assertEquals( + "The toNodeName of inDegree should be the nodeName of the node: A, inDegree: DAGEdge(fromNodeName=null, toNodeName=B)", + illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_NiceInDegree() { + assertDoesNotThrow(() -> DAGNode.builder() + .nodeName("A") + .inDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName(null) + .toNodeName("A") + .build())) + .outDegrees(new ArrayList<>()) + .build()); + } + + @Test + void buildDAGNode_BadOutDegree() { + IllegalArgumentException illegalArgumentException = assertThrows(IllegalArgumentException.class, + () -> DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName("B") + .toNodeName(null) + .build())) + .build()); + assertEquals( + "The fromNodeName of outDegree should be the nodeName of the node: A, outDegree: DAGEdge(fromNodeName=B, toNodeName=null)", + illegalArgumentException.getMessage()); + } + + @Test + void buildDAGNode_NiceOutDegree() { + assertDoesNotThrow(() -> DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(Lists.newArrayList(DAGEdge.builder() + .fromNodeName("A") + .toNodeName(null) + .build())) + .build()); + } + + @Test + void buildDAGNode_NotSkip() { + DAGNode dagNode = DAGNode.builder() + .nodeName("A") + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build(); + assertFalse(dagNode.isSkip()); + } + + @Test + void buildDAGNode_Skip() { + DAGNode dagNode = DAGNode.builder() + .nodeName("A") + .skip(true) + .inDegrees(new ArrayList<>()) + .outDegrees(new ArrayList<>()) + .build(); + assertTrue(dagNode.isSkip()); + } + +} diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilderTest.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilderTest.java new file mode 100644 index 0000000000..e37a73f32a --- /dev/null +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/dag/WorkflowDAGBuilderTest.java @@ -0,0 +1,260 @@ +package org.apache.dolphinscheduler.workflow.engine.dag; + +import static com.google.common.collect.Lists.newArrayList; +import static org.apache.dolphinscheduler.workflow.engine.assertions.WorkflowDAGAssertion.NodeAssertion.node; +import static org.apache.dolphinscheduler.workflow.engine.assertions.WorkflowDAGAssertion.workflowDag; + +import org.junit.jupiter.api.Test; + +class WorkflowDAGBuilderTest { + + /** + * Test DAG with single node: + *

    +     *     {@code
    +     *         Node(A)
    +     *     }
    +     *  
    + */ + @Test + void build_SingleTaskNode() { + String nodeName = "A"; + DAGNodeDefinition dagNodeDefinition = dagNode(nodeName); + WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder() + .addTaskNode(dagNodeDefinition) + .build(); + workflowDag(workflowDAG) + .nodeAssertion(node(nodeName).exist().noEdge()) + .doAssertion(); + } + + /** + * Test DAG with multiple nodes: + *
    +     *     {@code
    +     *        Node(A)
    +     *        Node(B)
    +     *        Node(C)
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_MULTIPLE_NODE() {
    +        String nodeNameA = "A";
    +        DAGNodeDefinition taskNodeA = dagNode(nodeNameA);
    +        String nodeNameB = "B";
    +        DAGNodeDefinition taskNodeB = dagNode(nodeNameB);
    +        String nodeNameC = "C";
    +        DAGNodeDefinition taskNodeC = dagNode(nodeNameC);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNode(taskNodeA)
    +                .addTaskNode(taskNodeB)
    +                .addTaskNode(taskNodeC)
    +                .build();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA).exist().noEdge())
    +                .nodeAssertion(node(nodeNameB).exist().noEdge())
    +                .nodeAssertion(node(nodeNameC).exist().noEdge())
    +                .doAssertion();
    +    }
    +
    +    /**
    +     * Test DAG with multiple nodes:
    +     * 
    +     *     {@code
    +     *          Node(A) -> Node(B1) -> Node(C1) -> Node(D)
    +     *                  -> Node(B2) -> Node(C2) ->
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_DAG() {
    +        String nodeNameA = "A";
    +        DAGNodeDefinition taskNodeA = dagNode(nodeNameA);
    +        String nodeNameB1 = "B1";
    +        DAGNodeDefinition taskNodeB1 = dagNode(nodeNameB1);
    +        String nodeNameB2 = "B2";
    +        DAGNodeDefinition taskNodeB2 = dagNode(nodeNameB2);
    +        String nodeNameC1 = "C1";
    +        DAGNodeDefinition taskNodeC1 = dagNode(nodeNameC1);
    +        String nodeNameC2 = "C2";
    +        DAGNodeDefinition taskNodeC2 = dagNode(nodeNameC2);
    +        String nodeNameD = "D";
    +        DAGNodeDefinition taskNodeD = dagNode(nodeNameD);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNode(taskNodeA)
    +                .addTaskNodes(newArrayList(taskNodeB1, taskNodeB2))
    +                .addTaskNodes(newArrayList(taskNodeC1, taskNodeC2))
    +                .addTaskNode(taskNodeD)
    +                .addTaskEdge(edge(null, taskNodeA))
    +                .addTaskEdge(edge(taskNodeA, taskNodeB1))
    +                .addTaskEdge(edge(taskNodeA, taskNodeB2))
    +                .addTaskEdge(edge(taskNodeB1, taskNodeC1))
    +                .addTaskEdge(edge(taskNodeB2, taskNodeC2))
    +                .addTaskEdge(edge(taskNodeC1, taskNodeD))
    +                .addTaskEdge(edge(taskNodeC2, taskNodeD))
    +                .addTaskEdge(edge(taskNodeD, null))
    +                .build();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB1, nodeNameB2)))
    +                .nodeAssertion(node(nodeNameB1)
    +                        .exist()
    +                        .inDegrees(nodeNameA)
    +                        .outDegrees(nodeNameC1))
    +                .nodeAssertion(node(nodeNameB2)
    +                        .exist()
    +                        .inDegrees(nodeNameA)
    +                        .outDegrees(nodeNameC2))
    +                .nodeAssertion(node(nodeNameC1)
    +                        .exist()
    +                        .inDegrees(nodeNameB1)
    +                        .outDegrees(nodeNameD))
    +                .nodeAssertion(node(nodeNameC2)
    +                        .exist()
    +                        .inDegrees(nodeNameB2)
    +                        .outDegrees(nodeNameD))
    +                .nodeAssertion(node(nodeNameD)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC1, nodeNameC2))
    +                        .noOutDegree())
    +                .doAssertion();
    +    }
    +
    +    /**
    +     * Test DAG with multiple sub dags:
    +     * 
    +     *     {@code
    +     *          Node(A1) -> Node(B1) -> Node(C1) -> Node(D1)
    +     *                  -> Node(B2) -> Node(C2) ->
    +     *
    +     *          Node(A2) -> Node(B3) -> Node(C3) -> Node(D2)
    +     *                   -> Node(B4) -> Node(C4) ->
    +     *     }
    +     * 
    +     */
    +    @Test
    +    void build_MULTIPLE_SUB_DAG() {
    +        String nodeNameA1 = "A1";
    +        DAGNodeDefinition taskNodeA1 = dagNode(nodeNameA1);
    +        String nodeNameA2 = "A2";
    +        DAGNodeDefinition taskNodeA2 = dagNode(nodeNameA2);
    +
    +        String nodeNameB1 = "B1";
    +        DAGNodeDefinition taskNodeB1 = dagNode(nodeNameB1);
    +        String nodeNameB2 = "B2";
    +        DAGNodeDefinition taskNodeB2 = dagNode(nodeNameB2);
    +        String nodeNameB3 = "B3";
    +        DAGNodeDefinition taskNodeB3 = dagNode(nodeNameB3);
    +        String nodeNameB4 = "B4";
    +        DAGNodeDefinition taskNodeB4 = dagNode(nodeNameB4);
    +
    +        String nodeNameC1 = "C1";
    +        DAGNodeDefinition taskNodeC1 = dagNode(nodeNameC1);
    +        String nodeNameC2 = "C2";
    +        DAGNodeDefinition taskNodeC2 = dagNode(nodeNameC2);
    +        String nodeNameC3 = "C3";
    +        DAGNodeDefinition taskNodeC3 = dagNode(nodeNameC3);
    +        String nodeNameC4 = "C4";
    +        DAGNodeDefinition taskNodeC4 = dagNode(nodeNameC4);
    +
    +        String nodeNameD1 = "D1";
    +        DAGNodeDefinition taskNodeD1 = dagNode(nodeNameD1);
    +        String nodeNameD2 = "D2";
    +        DAGNodeDefinition taskNodeD2 = dagNode(nodeNameD2);
    +
    +        WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
    +                .addTaskNodes(newArrayList(taskNodeA1, taskNodeA2))
    +                .addTaskNodes(newArrayList(taskNodeB1, taskNodeB2, taskNodeB3, taskNodeB4))
    +                .addTaskNodes(newArrayList(taskNodeC1, taskNodeC2, taskNodeC3, taskNodeC4))
    +                .addTaskNodes(newArrayList(taskNodeD1, taskNodeD2))
    +                .addTaskEdge(edge(null, taskNodeA1))
    +                .addTaskEdge(edge(taskNodeA1, taskNodeB1))
    +                .addTaskEdge(edge(taskNodeA1, taskNodeB2))
    +                .addTaskEdge(edge(taskNodeB1, taskNodeC1))
    +                .addTaskEdge(edge(taskNodeB2, taskNodeC2))
    +                .addTaskEdge(edge(taskNodeC1, taskNodeD1))
    +                .addTaskEdge(edge(taskNodeC2, taskNodeD1))
    +                .addTaskEdge(edge(taskNodeD1, null))
    +                .addTaskEdge(edge(null, taskNodeA2))
    +                .addTaskEdge(edge(taskNodeA2, taskNodeB3))
    +                .addTaskEdge(edge(taskNodeA2, taskNodeB4))
    +                .addTaskEdge(edge(taskNodeB3, taskNodeC3))
    +                .addTaskEdge(edge(taskNodeB4, taskNodeC4))
    +                .addTaskEdge(edge(taskNodeC3, taskNodeD2))
    +                .addTaskEdge(edge(taskNodeC4, taskNodeD2))
    +                .addTaskEdge(edge(taskNodeD2, null))
    +                .build();
    +
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA1)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB1, nodeNameB2)))
    +                .nodeAssertion(node(nodeNameB1)
    +                        .exist()
    +                        .inDegrees(nodeNameA1)
    +                        .outDegrees(nodeNameC1))
    +                .nodeAssertion(node(nodeNameB2)
    +                        .exist()
    +                        .inDegrees(nodeNameA1)
    +                        .outDegrees(nodeNameC2))
    +                .nodeAssertion(node(nodeNameC1)
    +                        .exist()
    +                        .inDegrees(nodeNameB1)
    +                        .outDegrees(nodeNameD1))
    +                .nodeAssertion(node(nodeNameC2)
    +                        .exist()
    +                        .inDegrees(nodeNameB2)
    +                        .outDegrees(nodeNameD1))
    +                .nodeAssertion(node(nodeNameD1)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC1, nodeNameC2))
    +                        .noOutDegree())
    +                .doAssertion();
    +        workflowDag(workflowDAG)
    +                .nodeAssertion(node(nodeNameA2)
    +                        .exist()
    +                        .noInDegree()
    +                        .outDegrees(newArrayList(nodeNameB3, nodeNameB4)))
    +                .nodeAssertion(node(nodeNameB3)
    +                        .exist()
    +                        .inDegrees(nodeNameA2)
    +                        .outDegrees(nodeNameC3))
    +                .nodeAssertion(node(nodeNameB4)
    +                        .exist()
    +                        .inDegrees(nodeNameA2)
    +                        .outDegrees(nodeNameC4))
    +                .nodeAssertion(node(nodeNameC3)
    +                        .exist()
    +                        .inDegrees(nodeNameB3)
    +                        .outDegrees(nodeNameD2))
    +                .nodeAssertion(node(nodeNameC4)
    +                        .exist()
    +                        .inDegrees(nodeNameB4)
    +                        .outDegrees(nodeNameD2))
    +                .nodeAssertion(node(nodeNameD2)
    +                        .exist()
    +                        .inDegrees(newArrayList(nodeNameC3, nodeNameC4))
    +                        .noOutDegree())
    +                .doAssertion();
    +    }
    +
    +    private DAGNodeDefinition dagNode(String nodeName) {
    +        DAGNodeDefinition dagNodeDefinition = new DAGNodeDefinition();
    +        dagNodeDefinition.setNodeName(nodeName);
    +        return dagNodeDefinition;
    +    }
    +
    +    private DAGEdge edge(DAGNodeDefinition from, DAGNodeDefinition to) {
    +        DAGEdge dagEdge = new DAGEdge();
    +        dagEdge.setFromNodeName(from == null ? null : from.getNodeName());
    +        dagEdge.setToNodeName(to == null ? null : to.getNodeName());
    +        return dagEdge;
    +    }
    +
    +}
    diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineIT.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineIT.java
    new file mode 100644
    index 0000000000..0ba306f3cc
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineIT.java
    @@ -0,0 +1,24 @@
    +/*
    + * 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.workflow.engine.engine;
    +
    +public class WorkflowEngineIT {
    +
    +    private WorkflowEngine workflowEngine;
    +
    +}
    diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineTest.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineTest.java
    new file mode 100644
    index 0000000000..6e35706549
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/engine/WorkflowEngineTest.java
    @@ -0,0 +1,105 @@
    +package org.apache.dolphinscheduler.workflow.engine.engine;
    +
    +import static org.apache.dolphinscheduler.workflow.engine.assertions.WorkflowExecutionRunnableAssertions.workflowExecutionRunnable;
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertThrows;
    +
    +import org.apache.dolphinscheduler.workflow.engine.assertions.SingletonWorkflowExecuteRunnableRepositoryAssertions;
    +import org.apache.dolphinscheduler.workflow.engine.event.WorkflowOperationEvent;
    +import org.apache.dolphinscheduler.workflow.engine.exception.WorkflowExecuteRunnableNotFoundException;
    +import org.apache.dolphinscheduler.workflow.engine.workflow.IWorkflowExecutionRunnable;
    +import org.apache.dolphinscheduler.workflow.engine.workflow.MockWorkflowExecutionRunnableFactory;
    +import org.apache.dolphinscheduler.workflow.engine.workflow.SingletonWorkflowExecuteRunnableRepository;
    +
    +import org.junit.jupiter.api.AfterEach;
    +import org.junit.jupiter.api.BeforeEach;
    +import org.junit.jupiter.api.Test;
    +
    +class WorkflowEngineTest {
    +
    +    private WorkflowEngine workflowEngine;
    +
    +    @BeforeEach
    +    public void before() {
    +        workflowEngine = new WorkflowEngine(SingletonWorkflowExecuteRunnableRepository.getInstance());
    +    }
    +
    +    @AfterEach
    +    public void after() {
    +        SingletonWorkflowExecuteRunnableRepository.getInstance().clear();
    +    }
    +
    +    @Test
    +    void triggerWorkflow() {
    +        IWorkflowExecutionRunnable mockWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecutionRunnable();
    +        Integer workflowInstanceId = mockWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +
    +        workflowEngine.triggerWorkflow(mockWorkflowExecuteRunnable);
    +        workflowExecutionRunnable(mockWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.triggerEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void pauseWorkflow_WorkflowNotExist() {
    +        WorkflowExecuteRunnableNotFoundException exception =
    +                assertThrows(WorkflowExecuteRunnableNotFoundException.class, () -> workflowEngine.pauseWorkflow(1));
    +        assertEquals("WorkflowExecuteRunnable not found: [id=1]", exception.getMessage());
    +    }
    +
    +    @Test
    +    void pauseWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable mockWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecutionRunnable();
    +        Integer workflowInstanceId =
    +                mockWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        SingletonWorkflowExecuteRunnableRepository.getInstance()
    +                .storeWorkflowExecutionRunnable(mockWorkflowExecuteRunnable);
    +
    +        workflowEngine.pauseWorkflow(workflowInstanceId);
    +        workflowExecutionRunnable(mockWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.pauseEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void killWorkflow_WorkflowNotExist() {
    +        WorkflowExecuteRunnableNotFoundException exception =
    +                assertThrows(WorkflowExecuteRunnableNotFoundException.class,
    +                        () -> workflowEngine.killWorkflow(1));
    +        assertEquals("WorkflowExecuteRunnable not found: [id=1]", exception.getMessage());
    +    }
    +
    +    @Test
    +    void killWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable mockWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecutionRunnable();
    +        Integer workflowInstanceId =
    +                mockWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        SingletonWorkflowExecuteRunnableRepository.getInstance()
    +                .storeWorkflowExecutionRunnable(mockWorkflowExecuteRunnable);
    +
    +        workflowEngine.killWorkflow(workflowInstanceId);
    +        workflowExecutionRunnable(mockWorkflowExecuteRunnable)
    +                .existEvent(WorkflowOperationEvent.killEvent(workflowInstanceId));
    +    }
    +
    +    @Test
    +    void finalizeWorkflow_WorkflowNotExist() {
    +        workflowEngine.finalizeWorkflow(-1);
    +    }
    +
    +    @Test
    +    void finalizeWorkflow_WorkflowExist() {
    +        IWorkflowExecutionRunnable emptyWorkflowExecuteRunnable =
    +                MockWorkflowExecutionRunnableFactory.createWorkflowExecutionRunnable();
    +        Integer workflowInstanceId =
    +                emptyWorkflowExecuteRunnable.getWorkflowExecutionContext().getWorkflowInstanceId();
    +        SingletonWorkflowExecuteRunnableRepository.getInstance()
    +                .storeWorkflowExecutionRunnable(emptyWorkflowExecuteRunnable);
    +        SingletonWorkflowExecuteRunnableRepositoryAssertions.existWorkflowExecutionRunnable(workflowInstanceId);
    +
    +        workflowEngine.finalizeWorkflow(workflowInstanceId);
    +        SingletonWorkflowExecuteRunnableRepositoryAssertions.notExistWorkflowExecutionRunnable(workflowInstanceId);
    +    }
    +
    +}
    diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventTest.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventTest.java
    new file mode 100644
    index 0000000000..8356fbb1ac
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/event/WorkflowOperationEventTest.java
    @@ -0,0 +1,30 @@
    +package org.apache.dolphinscheduler.workflow.engine.event;
    +
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +
    +import org.junit.jupiter.api.Test;
    +
    +class WorkflowOperationEventTest {
    +
    +    @Test
    +    void triggerEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.triggerEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.TRIGGER, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +
    +    @Test
    +    void pauseEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.pauseEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.PAUSE, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +
    +    @Test
    +    void killEvent() {
    +        WorkflowOperationEvent workflowOperationEvent = WorkflowOperationEvent.killEvent(1);
    +        assertEquals(1, workflowOperationEvent.getWorkflowInstanceId());
    +        assertEquals(WorkflowOperationType.KILL, workflowOperationEvent.getWorkflowOperationType());
    +    }
    +
    +}
    diff --git a/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/workflow/MockWorkflowExecutionRunnableFactory.java b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/workflow/MockWorkflowExecutionRunnableFactory.java
    new file mode 100644
    index 0000000000..94ecc00316
    --- /dev/null
    +++ b/dolphinscheduler-workflow-engine/src/test/java/org/apache/dolphinscheduler/workflow/engine/workflow/MockWorkflowExecutionRunnableFactory.java
    @@ -0,0 +1,46 @@
    +/*
    + * 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.workflow.engine.workflow;
    +
    +import org.apache.dolphinscheduler.workflow.engine.engine.DAGEngineFactory;
    +import org.apache.dolphinscheduler.workflow.engine.event.MemoryEventRepository;
    +
    +import org.apache.commons.lang3.RandomUtils;
    +
    +public class MockWorkflowExecutionRunnableFactory {
    +
    +    private static final ITaskExecutionContextFactory taskExecutionContextFactory = new TaskExecutionContextFactory();
    +    private static final ITaskExecutionRunnableFactory taskExecutionRunnableFactory =
    +            new TaskExecutionRunnableFactory(taskExecutionContextFactory);
    +    private static final DAGEngineFactory dagEngineFactory = new DAGEngineFactory(taskExecutionRunnableFactory);
    +    private static final WorkflowExecutionRunnableFactory workflowExecutionRunnableFactory =
    +            new WorkflowExecutionRunnableFactory(dagEngineFactory);
    +
    +    public static WorkflowExecutionRunnable createWorkflowExecutionRunnable() {
    +        int workflowInstanceId = RandomUtils.nextInt();
    +        String workflowInstanceName = "MockWorkflowInstance-" + workflowInstanceId;
    +        WorkflowInstance workflowInstance = WorkflowInstance.of(workflowInstanceId, workflowInstanceName);
    +        IWorkflowExecutionContext workflowExecutionContext = WorkflowExecutionContext.builder()
    +                .workflowInstance(workflowInstance)
    +                .eventRepository(MemoryEventRepository.newInstance())
    +                .build();
    +
    +        return workflowExecutionRunnableFactory.createWorkflowExecutionRunnable(workflowExecutionContext);
    +    }
    +
    +}
    diff --git a/pom.xml b/pom.xml
    index 4a71741f8e..1390f531a1 100755
    --- a/pom.xml
    +++ b/pom.xml
    @@ -57,6 +57,7 @@
             dolphinscheduler-storage-plugin
             dolphinscheduler-extract
             dolphinscheduler-dao-plugin
    +        dolphinscheduler-workflow-engine