Browse Source

Add dolphinscheduler-workflow-engine module

Wenjun Ruan 2 months ago
parent
commit
514d713aef
  1. 8
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
  2. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java
  3. 6
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java
  4. 71
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/BasicDAG.java
  5. 146
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAG.java
  6. 51
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEdge.java
  7. 148
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngine.java
  8. 32
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGEngineFactory.java
  9. 92
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/DAGNode.java
  10. 70
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngine.java
  11. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGEngineFactory.java
  12. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IDAGNodeAction.java
  13. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventRepositoryFactory.java
  14. 34
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IEventfulExecutionRunnable.java
  15. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContext.java
  16. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionContextFactory.java
  17. 59
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnable.java
  18. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableFactory.java
  19. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepository.java
  20. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/ITaskExecutionRunnableRepositoryFactory.java
  21. 25
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAG.java
  22. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowDAGFactory.java
  23. 57
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowEngine.java
  24. 46
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionContext.java
  25. 74
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAG.java
  26. 23
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionDAGFactory.java
  27. 54
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/IWorkflowExecutionRunnable.java
  28. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/MemoryEventRepositoryFactory.java
  29. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionContext.java
  30. 56
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnable.java
  31. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepository.java
  32. 32
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionRunnableRepositoryFactory.java
  33. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskExecutionValve.java
  34. 22
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/TaskIdentify.java
  35. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAG.java
  36. 125
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilder.java
  37. 65
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGFactory.java
  38. 82
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngine.java
  39. 52
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecuteRunnableRepository.java
  40. 47
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContext.java
  41. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionContextFactory.java
  42. 116
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAG.java
  43. 60
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionDAGFactory.java
  44. 64
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnable.java
  45. 40
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableFactory.java
  46. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dag/WorkflowIdentify.java
  47. 111
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventEngine.java
  48. 95
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventFirer.java
  49. 36
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/EventOperatorManager.java
  50. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IAsyncEvent.java
  51. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IDelayEvent.java
  52. 22
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEvent.java
  53. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventFirer.java
  54. 32
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperator.java
  55. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventOperatorManager.java
  56. 37
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IEventRepository.java
  57. 21
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ISyncEvent.java
  58. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEvent.java
  59. 22
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/ITaskEventOperator.java
  60. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEvent.java
  61. 23
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/IWorkflowEventOperator.java
  62. 60
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/MemoryEventRepository.java
  63. 36
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEvent.java
  64. 38
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskLogSendToRemoteEventOperator.java
  65. 46
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEvent.java
  66. 47
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationEventOperator.java
  67. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskOperationType.java
  68. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEvent.java
  69. 47
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/TaskSuccessEventOperator.java
  70. 35
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEvent.java
  71. 57
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFailedEventOperator.java
  72. 32
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEvent.java
  73. 42
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinalizeEventOperator.java
  74. 37
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowFinishEvent.java
  75. 51
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEvent.java
  76. 87
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventOperator.java
  77. 36
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationType.java
  78. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEvent.java
  79. 61
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTimeoutEventOperator.java
  80. 38
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEvent.java
  81. 44
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggerNextTaskEventOperator.java
  82. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEvent.java
  83. 63
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/events/WorkflowTriggeredEventOperator.java
  84. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableNotFoundException.java
  85. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowExecuteRunnableNotFoundException.java
  86. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DefaultTaskExecuteRunnable.java
  87. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
  88. 18
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java
  89. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/ExceptionUtils.java
  90. 121
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/DAGNodeTest.java
  91. 88
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionContextFactory.java
  92. 31
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/MockWorkflowExecutionRunnableFactory.java
  93. 171
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGAssertion.java
  94. 282
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowDAGBuilderTest.java
  95. 22
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineIT.java
  96. 114
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowEngineTest.java
  97. 48
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableAssertions.java
  98. 43
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableRepositoryAssertions.java
  99. 51
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dag/WorkflowExecutionRunnableTest.java
  100. 46
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/events/WorkflowOperationEventTest.java
  101. Some files were not shown because too many files have changed in this diff Show More

8
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 {

3
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessTaskRelationLogDao.java

@ -27,6 +27,9 @@ public interface ProcessTaskRelationLogDao extends IDao<ProcessTaskRelationLog>
List<ProcessTaskRelationLog> queryByWorkflowDefinitionCode(long workflowDefinitionCode);
List<ProcessTaskRelationLog> queryByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode,
int workflowDefinitionVerison);
void deleteByWorkflowDefinitionCode(long workflowDefinitionCode);
/**

6
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessTaskRelationLogDaoImpl.java

@ -44,6 +44,12 @@ public class ProcessTaskRelationLogDaoImpl extends BaseDao<ProcessTaskRelationLo
return mybatisMapper.queryByProcessCode(workflowDefinitionCode);
}
@Override
public List<ProcessTaskRelationLog> queryByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode,
int workflowDefinitionVerison) {
return mybatisMapper.queryByProcessCodeAndVersion(workflowDefinitionCode, workflowDefinitionVerison);
}
@Override
public void deleteByWorkflowDefinitionCode(long workflowDefinitionCode) {
mybatisMapper.deleteByWorkflowDefinitionCode(workflowDefinitionCode);

71
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<String, DAGNode> dagNodeMap;
@Override
public List<DAGNode> getDirectPostNodes(DAGNode dagNode) {
final String nodeName = dagNode.getNodeName();
if (!dagNodeMap.containsKey(nodeName)) {
return Collections.emptyList();
}
DAGNode node = dagNodeMap.get(nodeName);
List<DAGNode> dagNodes = new ArrayList<>();
for (DAGEdge edge : node.getOutDegrees()) {
if (dagNodeMap.containsKey(edge.getToNodeName())) {
dagNodes.add(dagNodeMap.get(edge.getToNodeName()));
}
}
return dagNodes;
}
@Override
public List<DAGNode> getDirectPreNodes(DAGNode dagNode) {
final String nodeName = dagNode.getNodeName();
if (!dagNodeMap.containsKey(nodeName)) {
return Collections.emptyList();
}
DAGNode node = dagNodeMap.get(nodeName);
List<DAGNode> 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);
}
}

146
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.
* <p>
* 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:
* <pre>
* {@code
* 1 -> 2 -> 3
* 4 -> 5
* 6
* }
* </pre>
* <li> The post node of 1 is 2.
* <li> The post node of 3 is empty.
* <li> 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<DAGNode> getDirectPostNodes(DAGNode dagNode);
/**
* Same with {@link #getDirectPostNodes(DAGNode)}.
* <p>
* 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<DAGNode> 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<String> 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:
* <pre>
* {@code
* 1 -> 2 -> 3
* 4 -> 5
* 6
* }
* </pre>
* <li> The pre node of 1 is empty.
* <li> The pre node of 3 is 2.
* <li> 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<DAGNode> getDirectPreNodes(DAGNode dagNode);
/**
* Same with {@link #getDirectPreNodes(DAGNode)}.
* <p>
* 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<DAGNode> 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<String> 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);
}

51
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.
* <p>
* 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.
* <p>
* 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;
}
}

148
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);
}
}

32
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);
}
}

92
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.
* <p>
* 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<DAGEdge> inDegrees;
private List<DAGEdge> outDegrees;
public DAGNode(String nodeName,
List<DAGEdge> inDegrees,
List<DAGEdge> outDegrees) {
this(nodeName, false, inDegrees, outDegrees);
}
public DAGNode(String nodeName,
boolean skip,
List<DAGEdge> inDegrees,
List<DAGEdge> 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;
}
}

70
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}.
* <p>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.
* <P> If there are no task after the given taskNode, will try to finish the WorkflowExecutionRunnable.
* <p> 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);
}

24
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);
}

35
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();
}

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java → 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();
}

34
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);
}
}

26
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();
}

24
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();
}

59
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);
}

24
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);
}

33
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<TaskExecutionRunnable> getActiveTaskExecutionRunnable();
void removeTaskExecutionRunnable(Integer taskInstanceId);
}

24
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();
}

25
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 {
}

35
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);
}

57
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);
}

46
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<String> getBeginNodeNames();
IEventRepository getEventRepository();
default int getWorkflowInstanceId() {
return getWorkflowInstance().getId();
}
default String getWorkflowInstanceName() {
return getWorkflowInstance().getName();
}
}

74
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<TaskExecutionRunnable> getActiveTaskExecutionRunnable();
/**
* Get the direct pre TaskExecutionRunnable of the given taskName.
*
* @param taskName task name.
* @return TaskExecutionRunnable
*/
List<TaskExecutionRunnable> 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);
}

23
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();
}

54
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();
}

35
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();
}
}

35
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;
}

56
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;
}
}

68
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<Integer, TaskExecutionRunnable> taskExecuteRunnableMap = new ConcurrentHashMap<>();
private final Map<String, TaskExecutionRunnable> 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<TaskExecutionRunnable> 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());
}
}
}

32
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();
}
}

27
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);
}

22
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 {
}

27
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 {
}

125
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.
* <p>
* Example:
* <pre>
* {@code
* WorkflowDAG workflowDAG = WorkflowDAGBuilder.newBuilder()
* .addTaskNode(taskNodeA)
* .addTaskNode(taskNodeB)
* .addTaskNode(taskNodeC)
* .addTaskEdge(edgeAB)
* .addTaskEdge(edgeBC)
* .build();
* }
* </pre>
*/
public class WorkflowDAGBuilder {
private final Map<String, DAGNode> taskNameMap;
private final Map<Long, DAGNode> taskCodeMap;
private WorkflowDAGBuilder() {
this.taskCodeMap = new HashMap<>();
this.taskNameMap = new HashMap<>();
}
public static WorkflowDAGBuilder newBuilder() {
return new WorkflowDAGBuilder();
}
public WorkflowDAGBuilder addTaskNodes(List<TaskDefinitionLog> 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<ProcessTaskRelationLog> 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();
}
}

65
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<TaskDefinitionLog> taskDefinitions = queryTaskNodes(workflowDefinitionCode, workflowDefinitionVersion);
List<ProcessTaskRelationLog> taskRelations = queryTaskEdges(workflowDefinitionCode, workflowDefinitionVersion);
return WorkflowDAGBuilder.newBuilder()
.addTaskNodes(taskDefinitions)
.addTaskEdges(taskRelations)
.build();
}
private List<TaskDefinitionLog> queryTaskNodes(Long workflowDefinitionCode, Integer workflowDefinitionVersion) {
return taskDefinitionLogDao.queryByWorkflowDefinitionCodeAndVersion(workflowDefinitionCode,
workflowDefinitionVersion);
}
private List<ProcessTaskRelationLog> queryTaskEdges(Long workflowDefinitionCode,
Integer workflowDefinitionVersion) {
return processTaskRelationLogDao.queryByWorkflowDefinitionCodeAndVersion(workflowDefinitionCode,
workflowDefinitionVersion);
}
}

82
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);
}
}

52
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<Integer, IWorkflowExecutionRunnable> 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<IWorkflowExecutionRunnable> getActiveWorkflowExecutionRunnable() {
return workflowExecutionRunnableMap.values();
}
public void removeWorkflowExecutionRunnable(Integer workflowInstanceId) {
workflowExecutionRunnableMap.remove(workflowInstanceId);
}
}

47
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<String> beginNodeNames;
private IWorkflowExecutionDAG workflowExecutionDAG;
private IEventRepository eventRepository;
}

31
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;
}
}

116
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<TaskExecutionRunnable> getActiveTaskExecutionRunnable() {
return new ArrayList<>(taskExecutionRunnableRepository.getActiveTaskExecutionRunnable());
}
@Override
public List<TaskExecutionRunnable> 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<DAGNode> 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<DAGNode> getDirectPostNodes(DAGNode dagNode) {
return workflowDAG.getDirectPostNodes(dagNode);
}
@Override
public List<DAGNode> getDirectPreNodes(DAGNode dagNode) {
return workflowDAG.getDirectPreNodes(dagNode);
}
@Override
public DAGNode getDAGNode(String nodeName) {
return workflowDAG.getDAGNode(nodeName);
}
}

60
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<TaskExecutionRunnable> loadTheHistoryTaskExecutionRunnable() {
return new ArrayList<>();
}
}

64
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();
}
}

40
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();
}
}

35
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;
}

111
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<Integer> 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<IWorkflowExecutionRunnable> 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();
}
}
}
}

95
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<IEvent> eventOperatorManager;
private final ThreadPoolExecutor eventFireThreadPool;
public EventFirer(IEventOperatorManager<IEvent> eventOperatorManager, MasterConfig masterConfig) {
this.eventOperatorManager = eventOperatorManager;
this.eventFireThreadPool =
ThreadUtils.newDaemonFixedThreadExecutor("EventFireThreadPool", masterConfig.getExecThreads());
}
@Override
public CompletableFuture<Integer> 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);
}
}

36
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<IEvent> {
@Override
public IEventOperator<IEvent> getEventOperator(IEvent event) {
return null;
}
}

24
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 {
}

26
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();
}

22
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 {
}

35
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<Integer> fireActiveEvents(IEventRepository eventRepository);
}

32
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<E> {
/**
* Handle the given event
*
* @param event event
*/
void handleEvent(E event);
}

33
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<E> {
/**
* Get the {@link IEventOperator} for the given event.
*
* @param event event
* @return event operator for the given event
*/
IEventOperator<E> getEventOperator(E event);
}

37
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<IEvent> getAllEvent();
}

21
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 {
}

26
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();
}

22
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<E extends ITaskEvent> extends IEventOperator<E> {
}

29
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();
}

23
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<E extends IWorkflowEvent>
extends
IEventOperator<E> {
}

60
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<IEvent> 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<IEvent> getAllEvent() {
return new ArrayList<>(eventQueue);
}
}

36
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;
}

38
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<TaskLogSendToRemoteEvent> {
@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());
}
}
}

46
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();
}
}

47
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<TaskOperationEvent> {
@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);
}
}
}

29
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,
;
}

33
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;
}

47
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<TaskSuccessEvent> {
@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;
}
}
}

35
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;
}

57
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<WorkflowFailedEvent> {
@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));
}
}

32
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;
}

42
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<WorkflowFinalizeEvent> {
@Autowired
private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository;
@Override
public void handleEvent(WorkflowFinalizeEvent event) {
Integer workflowInstanceId = event.getWorkflowInstanceId();
workflowExecuteRunnableRepository.removeWorkflowExecutionRunnable(workflowInstanceId);
}
}

37
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;
}

51
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);
}
}

87
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<WorkflowOperationEvent> {
@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();
}
}

36
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,
;
}

33
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;
}

61
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<WorkflowTimeoutEvent> {
@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);
}
}

38
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;
}

44
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<WorkflowTriggerNextTaskEvent> {
@Autowired
private WorkflowExecuteRunnableRepository workflowExecuteRunnableRepository;
@Override
public void handleEvent(WorkflowTriggerNextTaskEvent event) {
IWorkflowExecutionRunnable workflowExecutionRunnable =
workflowExecuteRunnableRepository.getWorkflowExecutionRunnableById(event.getWorkflowInstanceId());
workflowExecutionRunnable.getDagEngine().triggerNextTasks(event.getParentTaskNodeName());
}
}

33
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;
}

63
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<WorkflowTriggeredEvent> {
@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);
}
}

29
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 + "]");
}
}

30
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 + "]");
}
}

3
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);

3
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 =

18
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);
}
}

31
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;
}
}

121
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());
}
}

88
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;
}
}

31
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();
}
}

171
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<NodeAssertion> 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<String> postNodes;
/**
* whether the node has in degree
*/
private List<String> 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<String> 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<String> 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));
}
}
}
}

282
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:
* <pre>
* {@code
* Node(A)
* }
* </pre>
*/
@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:
* <pre>
* {@code
* Node(A)
* Node(B)
* Node(C)
* }
* <pre/>
*/
@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:
* <pre>
* {@code
* Node(A) -> Node(B1) -> Node(C1) -> Node(D)
* -> Node(B2) -> Node(C2) ->
* }
* <pre/>
*/
@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:
* <pre>
* {@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) ->
* }
* <pre/>
*/
@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;
}
}

22
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 {
}

114
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);
}
}

48
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);
}
}
}

43
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));
}
}

51
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() {
}
}

46
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());
}
}

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save