Browse Source

[DSIP-65] SubWorkflow logic task support failover/repeat running/pause/kill/recover (#16566)

dev
Wenjun Ruan 3 months ago committed by GitHub
parent
commit
3a24d1777b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 49
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  2. 39
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java
  3. 108
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RecoverFailureTaskCommandHandler.java
  4. 17
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java
  5. 17
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/WorkflowFailoverCommandHandler.java
  6. 86
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphTopologyLogicalVisitor.java
  7. 83
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/FailedRecoverTaskInstanceFactory.java
  8. 70
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/PauseRecoverTaskInstanceFactory.java
  9. 13
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java
  10. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverFailureTaskTrigger.java
  11. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverSuspendTaskTrigger.java
  12. 7
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java
  13. 6
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java
  14. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java
  15. 14
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java
  16. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java
  17. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java
  18. 11
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java
  19. 32
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
  20. 5
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
  21. 45
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java
  22. 107
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java
  23. 258
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
  24. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java
  25. 11
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java
  26. 40
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java
  27. 2
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java
  28. 16
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java
  29. 11
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java
  30. 2
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java
  31. 30
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java
  32. 87
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java
  33. 136
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverPauseIT.java
  34. 147
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverStopIT.java
  35. 79
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java
  36. 4
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java
  37. 135
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java
  38. 22
      dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml
  39. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml
  40. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml
  41. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml
  42. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml
  43. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml
  44. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml
  45. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml
  46. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml
  47. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml
  48. 22
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml
  49. 22
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml
  50. 22
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml
  51. 22
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml
  52. 22
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml
  53. 22
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml
  54. 22
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml
  55. 22
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml
  56. 111
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_sub_workflow_task_success.yaml
  57. 22
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml
  58. 22
      dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml
  59. 111
      dolphinscheduler-master/src/test/resources/it/recover_paused/workflow_with_sub_workflow_task_success.yaml
  60. 111
      dolphinscheduler-master/src/test/resources/it/recover_stopped/workflow_with_sub_workflow_task_success.yaml
  61. 22
      dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml
  62. 22
      dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml
  63. 22
      dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml
  64. 22
      dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml
  65. 24
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml
  66. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml
  67. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml
  68. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml
  69. 91
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_failed.yaml
  70. 91
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_success.yaml
  71. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml
  72. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml
  73. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml
  74. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml
  75. 22
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml
  76. 22
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml
  77. 22
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml
  78. 111
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_sub_workflow_task_success.yaml
  79. 22
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml

49
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java

@ -20,13 +20,10 @@ package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskExecuteType; import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
import java.io.Serializable; import java.io.Serializable;
import java.util.Date; import java.util.Date;
import java.util.concurrent.TimeUnit;
import lombok.Data; import lombok.Data;
@ -273,50 +270,4 @@ public class TaskInstance implements Serializable {
this.executePath = executePath; this.executePath = executePath;
} }
public boolean isTaskComplete() {
return this.getState().isSuccess()
|| this.getState().isKill()
|| (this.getState().isFailure() && !taskCanRetry())
|| this.getState().isForceSuccess();
}
public boolean isFirstRun() {
return endTime == null;
}
/**
* determine if a task instance can retry
* if subProcess,
*
* @return can try result
*/
public boolean taskCanRetry() {
if (TaskTypeUtils.isSubWorkflowTask(getTaskType())) {
return false;
}
if (this.getState() == TaskExecutionStatus.NEED_FAULT_TOLERANCE) {
return true;
}
return this.getState() == TaskExecutionStatus.FAILURE && (this.getRetryTimes() < this.getMaxRetryTimes());
}
/**
* whether the retry interval is timed out
*
* @return Boolean
*/
public boolean retryTaskIntervalOverTime() {
if (getState() != TaskExecutionStatus.FAILURE) {
return true;
}
if (getMaxRetryTimes() == 0 || getRetryInterval() == 0) {
return true;
}
Date now = new Date();
long failedTimeInterval = DateUtils.differSec(now, getEndTime());
// task retry does not over time, return false
return TimeUnit.MINUTES.toSeconds(getRetryInterval()) < failedTimeInterval;
}
} }

39
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java

@ -17,15 +17,19 @@
package org.apache.dolphinscheduler.dao.entity; package org.apache.dolphinscheduler.dao.entity;
import java.util.Objects; import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data; import lombok.Data;
import lombok.NoArgsConstructor;
import com.baomidou.mybatisplus.annotation.IdType; import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId; import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName; import com.baomidou.mybatisplus.annotation.TableName;
@Data @Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
@TableName("t_ds_relation_process_instance") @TableName("t_ds_relation_process_instance")
public class WorkflowInstanceRelation { public class WorkflowInstanceRelation {
@ -50,35 +54,4 @@ public class WorkflowInstanceRelation {
*/ */
private int processInstanceId; private int processInstanceId;
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
WorkflowInstanceRelation that = (WorkflowInstanceRelation) o;
if (!Objects.equals(id, that.id)) {
return false;
}
if (parentProcessInstanceId != that.parentProcessInstanceId) {
return false;
}
if (parentTaskInstanceId != that.parentTaskInstanceId) {
return false;
}
return processInstanceId == that.processInstanceId;
}
@Override
public int hashCode() {
int result = id;
result = 31 * result + parentProcessInstanceId;
result = 31 * result + parentTaskInstanceId;
result = 31 * result + processInstanceId;
return result;
}
} }

108
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RecoverFailureTaskCommandHandler.java

@ -28,14 +28,15 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.engine.TaskGroupCoordinator; import org.apache.dolphinscheduler.server.master.engine.TaskGroupCoordinator;
import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph; import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphBfsVisitor; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphTopologyLogicalVisitor;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskInstanceFactories;
import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory; import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Date; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -71,6 +72,9 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
@Autowired @Autowired
private ApplicationContext applicationContext; private ApplicationContext applicationContext;
@Autowired
private TaskInstanceFactories taskInstanceFactories;
/** /**
* Generate the recover workflow instance. * Generate the recover workflow instance.
* <p> Will use the origin workflow instance, but will update the following fields. Need to note we cannot not * <p> Will use the origin workflow instance, but will update the following fields. Need to note we cannot not
@ -94,10 +98,6 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
workflowInstance.setVarPool(null); workflowInstance.setVarPool(null);
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name()); workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());
workflowInstance.setCommandType(command.getCommandType()); workflowInstance.setCommandType(command.getCommandType());
workflowInstance.setStartTime(new Date());
workflowInstance.setRestartTime(new Date());
workflowInstance.setEndTime(null);
workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1);
workflowInstanceDao.updateById(workflowInstance); workflowInstanceDao.updateById(workflowInstance);
workflowExecuteContextBuilder.setWorkflowInstance(workflowInstance); workflowExecuteContextBuilder.setWorkflowInstance(workflowInstance);
@ -133,13 +133,14 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors); workflowExecutionGraph.addEdge(task, successors);
}; };
final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder() final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
.taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType()) WorkflowGraphTopologyLogicalVisitor.builder()
.onWorkflowGraph(workflowGraph) .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
.fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder)) .onWorkflowGraph(workflowGraph)
.doVisitFunction(taskExecutionRunnableCreator) .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
.build(); .doVisitFunction(taskExecutionRunnableCreator)
workflowGraphBfsVisitor.visit(); .build();
workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph); workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
} }
@ -155,42 +156,52 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
.stream() .stream()
.collect(Collectors.toMap(TaskInstance::getName, Function.identity())); .collect(Collectors.toMap(TaskInstance::getName, Function.identity()));
final Set<String> needRecreateTasks = taskInstanceMap.values()
.stream()
.filter(this::isTaskNeedRecreate)
.map(TaskInstance::getName)
.collect(Collectors.toSet());
final IWorkflowGraph workflowGraph = workflowExecuteContextBuilder.getWorkflowGraph(); final IWorkflowGraph workflowGraph = workflowExecuteContextBuilder.getWorkflowGraph();
final Set<String> needRecoverTasks = new HashSet<>();
final Set<String> markInvalidTasks = new HashSet<>();
final BiConsumer<String, Set<String>> historyTaskInstanceMarker = (task, successors) -> { final BiConsumer<String, Set<String>> historyTaskInstanceMarker = (task, successors) -> {
boolean isTaskNeedRecreate = needRecreateTasks.contains(task) || workflowGraph.getPredecessors(task) // If the parent is need recover
.stream() // Then the task should mark as invalid, and it's child should be mark as invalidated.
.anyMatch(needRecreateTasks::contains); if (markInvalidTasks.contains(task)) {
// If the task instance need to be recreated, then will mark the task instance invalid.
// and the TaskExecutionRunnable will not contain the task instance.
if (isTaskNeedRecreate) {
needRecreateTasks.add(task);
if (taskInstanceMap.containsKey(task)) { if (taskInstanceMap.containsKey(task)) {
taskInstanceDao.markTaskInstanceInvalid(Lists.newArrayList(taskInstanceMap.get(task))); taskInstanceDao.markTaskInstanceInvalid(Lists.newArrayList(taskInstanceMap.get(task)));
taskInstanceMap.remove(task); taskInstanceMap.remove(task);
} }
markInvalidTasks.addAll(successors);
return;
} }
// If the task instance need to be recovered, then will mark the task instance to submit.
// and the TaskExecutionRunnable will contain the task instance and pass the creation step. final TaskInstance taskInstance = taskInstanceMap.get(task);
if (isTaskNeedRecover(taskInstanceMap.get(task))) { if (taskInstance == null) {
final TaskInstance taskInstance = taskInstanceMap.get(task); return;
taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); }
taskInstanceDao.upsertTaskInstance(taskInstance);
if (isTaskNeedRecreate(taskInstance) || isTaskCanRecover(taskInstance)) {
needRecoverTasks.add(task);
markInvalidTasks.addAll(successors);
} }
}; };
final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder() final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
.onWorkflowGraph(workflowGraph) WorkflowGraphTopologyLogicalVisitor.builder()
.taskDependType(workflowInstance.getTaskDependType()) .onWorkflowGraph(workflowGraph)
.fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder)) .taskDependType(workflowInstance.getTaskDependType())
.doVisitFunction(historyTaskInstanceMarker) .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
.build(); .doVisitFunction(historyTaskInstanceMarker)
workflowGraphBfsVisitor.visit(); .build();
workflowGraphTopologyLogicalVisitor.visit();
for (String task : needRecoverTasks) {
final TaskInstance taskInstance = taskInstanceMap.get(task);
if (isTaskCanRecover(taskInstance)) {
taskInstanceMap.put(task, createRecoverTaskInstance(taskInstance));
continue;
}
if (isTaskNeedRecreate(taskInstance)) {
taskInstanceMap.put(task, createRecreatedTaskInstance(taskInstance));
}
}
return new ArrayList<>(taskInstanceMap.values()); return new ArrayList<>(taskInstanceMap.values());
} }
@ -199,17 +210,34 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
* <p> If the task state is FAILURE and KILL, then will mark the task invalid and recreate the task. * <p> If the task state is FAILURE and KILL, then will mark the task invalid and recreate the task.
*/ */
private boolean isTaskNeedRecreate(final TaskInstance taskInstance) { private boolean isTaskNeedRecreate(final TaskInstance taskInstance) {
if (taskInstance == null) {
return false;
}
return taskInstance.getState() == TaskExecutionStatus.FAILURE return taskInstance.getState() == TaskExecutionStatus.FAILURE
|| taskInstance.getState() == TaskExecutionStatus.KILL; || taskInstance.getState() == TaskExecutionStatus.KILL;
} }
private boolean isTaskNeedRecover(final TaskInstance taskInstance) { private TaskInstance createRecreatedTaskInstance(final TaskInstance taskInstance) {
return taskInstanceFactories.failedRecoverTaskInstanceFactory()
.builder()
.withTaskInstance(taskInstance)
.build();
}
private boolean isTaskCanRecover(final TaskInstance taskInstance) {
if (taskInstance == null) { if (taskInstance == null) {
return false; return false;
} }
return taskInstance.getState() == TaskExecutionStatus.PAUSE; return taskInstance.getState() == TaskExecutionStatus.PAUSE;
} }
private TaskInstance createRecoverTaskInstance(final TaskInstance taskInstance) {
return taskInstanceFactories.pauseRecoverTaskInstanceFactory()
.builder()
.withTaskInstance(taskInstance)
.build();
}
@Override @Override
public CommandType commandType() { public CommandType commandType() {
return CommandType.START_FAILURE_TASK_PROCESS; return CommandType.START_FAILURE_TASK_PROCESS;

17
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java

@ -31,7 +31,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph; import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphBfsVisitor; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphTopologyLogicalVisitor;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
@ -107,13 +107,14 @@ public class RunWorkflowCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors); workflowExecutionGraph.addEdge(task, successors);
}; };
final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder() final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
.taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType()) WorkflowGraphTopologyLogicalVisitor.builder()
.onWorkflowGraph(workflowGraph) .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
.fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder)) .onWorkflowGraph(workflowGraph)
.doVisitFunction(taskExecutionRunnableCreator) .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
.build(); .doVisitFunction(taskExecutionRunnableCreator)
workflowGraphBfsVisitor.visit(); .build();
workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph); workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
} }

17
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/WorkflowFailoverCommandHandler.java

@ -28,7 +28,7 @@ import org.apache.dolphinscheduler.extract.master.command.WorkflowFailoverComman
import org.apache.dolphinscheduler.server.master.engine.TaskGroupCoordinator; import org.apache.dolphinscheduler.server.master.engine.TaskGroupCoordinator;
import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph; import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphBfsVisitor; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphTopologyLogicalVisitor;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory; import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory;
@ -129,13 +129,14 @@ public class WorkflowFailoverCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors); workflowExecutionGraph.addEdge(task, successors);
}; };
final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder() final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
.taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType()) WorkflowGraphTopologyLogicalVisitor.builder()
.onWorkflowGraph(workflowGraph) .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
.fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder)) .onWorkflowGraph(workflowGraph)
.doVisitFunction(taskExecutionRunnableCreator) .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
.build(); .doVisitFunction(taskExecutionRunnableCreator)
workflowGraphBfsVisitor.visit(); .build();
workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph); workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
} }

86
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphBfsVisitor.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphTopologyLogicalVisitor.java

@ -20,27 +20,31 @@ package org.apache.dolphinscheduler.server.master.engine.graph;
import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.common.enums.TaskDependType; import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.CollectionUtils;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class WorkflowGraphBfsVisitor { import com.google.common.collect.Sets;
private IWorkflowGraph workflowGraph; public class WorkflowGraphTopologyLogicalVisitor {
private TaskDependType taskDependType; private final IWorkflowGraph workflowGraph;
private Set<String> startNodes; private final TaskDependType taskDependType;
private BiConsumer<String, Set<String>> visitFunction; private final Set<String> startNodes;
private WorkflowGraphBfsVisitor(WorkflowGraphBfsVisitorBuilder workflowGraphBfsVisitorBuilder) { private final BiConsumer<String, Set<String>> visitFunction;
private WorkflowGraphTopologyLogicalVisitor(WorkflowGraphBfsVisitorBuilder workflowGraphBfsVisitorBuilder) {
this.taskDependType = workflowGraphBfsVisitorBuilder.taskDependType; this.taskDependType = workflowGraphBfsVisitorBuilder.taskDependType;
this.workflowGraph = checkNotNull(workflowGraphBfsVisitorBuilder.workflowGraph); this.workflowGraph = checkNotNull(workflowGraphBfsVisitorBuilder.workflowGraph);
this.visitFunction = checkNotNull(workflowGraphBfsVisitorBuilder.visitFunction); this.visitFunction = checkNotNull(workflowGraphBfsVisitorBuilder.visitFunction);
@ -72,60 +76,76 @@ public class WorkflowGraphBfsVisitor {
} }
/** /**
* visit start nodes only * Visit start nodes only.
*/ */
private void visitStartNodesOnly() { private void visitStartNodesOnly() {
startNodes.forEach(startNode -> { doVisitationInSubGraph(Sets.newHashSet(startNodes));
final Set<String> realSuccessors = workflowGraph.getSuccessors(startNode)
.stream()
.filter(startNode::contains)
.collect(Collectors.toSet());
visitFunction.accept(startNode, realSuccessors);
});
} }
/** /**
* Find the graph nodes that can be reached to the start nodes * Find the graph nodes that can be reached to the start nodes, and then do visitation with topology logical.
*/ */
private void visitToStartNodes() { private void visitToStartNodes() {
final LinkedList<String> bootstrapTaskCodes = new LinkedList<>(startNodes); final LinkedList<String> bootstrapTaskCodes = new LinkedList<>(startNodes);
final Set<String> visited = new HashSet<>(); final Set<String> subGraphNodes = new HashSet<>();
while (!bootstrapTaskCodes.isEmpty()) { while (!bootstrapTaskCodes.isEmpty()) {
String taskName = bootstrapTaskCodes.removeFirst(); String taskName = bootstrapTaskCodes.removeFirst();
if (visited.contains(taskName)) { if (subGraphNodes.contains(taskName)) {
continue; continue;
} }
visited.add(taskName); subGraphNodes.add(taskName);
final Set<String> successors = workflowGraph.getPredecessors(taskName); final Set<String> successors = workflowGraph.getPredecessors(taskName);
bootstrapTaskCodes.addAll(successors); bootstrapTaskCodes.addAll(successors);
} }
visited.forEach(taskName -> { doVisitationInSubGraph(subGraphNodes);
Set<String> realSuccessors = workflowGraph.getSuccessors(taskName)
.stream()
.filter(visited::contains)
.collect(Collectors.toSet());
visitFunction.accept(taskName, realSuccessors);
});
} }
/** /**
* Find the graph nodes that can be reached from the start nodes * Find the graph nodes that can be reached from the start nodes, and then do visitation with topology logical.
*/ */
private void visitFromStartNodes() { private void visitFromStartNodes() {
final LinkedList<String> bootstrapTaskCodes = new LinkedList<>(startNodes); final LinkedList<String> bootstrapTaskCodes = new LinkedList<>(startNodes);
final Set<String> visited = new HashSet<>(); final Set<String> subGraphNodes = new HashSet<>();
while (!bootstrapTaskCodes.isEmpty()) { while (!bootstrapTaskCodes.isEmpty()) {
String taskName = bootstrapTaskCodes.removeFirst(); String taskName = bootstrapTaskCodes.removeFirst();
if (visited.contains(taskName)) { if (subGraphNodes.contains(taskName)) {
continue; continue;
} }
visited.add(taskName); subGraphNodes.add(taskName);
final Set<String> successors = workflowGraph.getSuccessors(taskName); final Set<String> successors = workflowGraph.getSuccessors(taskName);
visitFunction.accept(taskName, successors);
bootstrapTaskCodes.addAll(successors); bootstrapTaskCodes.addAll(successors);
} }
doVisitationInSubGraph(subGraphNodes);
}
private void doVisitationInSubGraph(Set<String> subGraphNodes) {
// visit from the workflow graph by topology
// If the node is not in the subGraph, then skip it.
Map<String, Integer> inDegreeMap = workflowGraph.getAllTaskNodes()
.stream()
.collect(Collectors.toMap(TaskDefinition::getName,
taskDefinition -> workflowGraph.getPredecessors(taskDefinition.getName()).size()));
final LinkedList<String> bootstrapTaskCodes = inDegreeMap
.entrySet()
.stream()
.filter(entry -> entry.getValue() == 0)
.map(Map.Entry::getKey)
.collect(Collectors.toCollection(LinkedList::new));
while (!bootstrapTaskCodes.isEmpty()) {
String taskName = bootstrapTaskCodes.removeFirst();
if (inDegreeMap.get(taskName) > 0) {
continue;
}
final Set<String> successors = workflowGraph.getSuccessors(taskName);
if (subGraphNodes.contains(taskName)) {
visitFunction.accept(taskName, successors);
}
for (String successor : successors) {
inDegreeMap.put(successor, inDegreeMap.get(successor) - 1);
}
bootstrapTaskCodes.addAll(successors);
}
} }
public static class WorkflowGraphBfsVisitorBuilder { public static class WorkflowGraphBfsVisitorBuilder {
@ -158,8 +178,8 @@ public class WorkflowGraphBfsVisitor {
return this; return this;
} }
public WorkflowGraphBfsVisitor build() { public WorkflowGraphTopologyLogicalVisitor build() {
return new WorkflowGraphBfsVisitor(this); return new WorkflowGraphTopologyLogicalVisitor(this);
} }
} }
} }

83
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/FailedRecoverTaskInstanceFactory.java

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.task.runnable;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import java.util.Date;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import org.springframework.transaction.annotation.Transactional;
@Component
public class FailedRecoverTaskInstanceFactory
extends
AbstractTaskInstanceFactory<FailedRecoverTaskInstanceFactory.FailedRecoverTaskInstanceBuilder> {
@Autowired
private TaskInstanceDao taskInstanceDao;
@Override
public FailedRecoverTaskInstanceFactory.FailedRecoverTaskInstanceBuilder builder() {
return new FailedRecoverTaskInstanceBuilder(this);
}
@Transactional
@Override
public TaskInstance createTaskInstance(FailedRecoverTaskInstanceBuilder builder) {
final TaskInstance needRecoverTaskInstance = builder.needRecoverTaskInstance;
final TaskInstance taskInstance = cloneTaskInstance(needRecoverTaskInstance);
taskInstance.setId(null);
taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
taskInstance.setHost(null);
taskInstance.setVarPool(null);
taskInstance.setSubmitTime(new Date());
taskInstance.setLogPath(null);
taskInstance.setExecutePath(null);
taskInstanceDao.insert(taskInstance);
needRecoverTaskInstance.setFlag(Flag.NO);
taskInstanceDao.updateById(needRecoverTaskInstance);
return taskInstance;
}
public static class FailedRecoverTaskInstanceBuilder implements ITaskInstanceFactory.ITaskInstanceBuilder {
private final FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory;
private TaskInstance needRecoverTaskInstance;
public FailedRecoverTaskInstanceBuilder(FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory) {
this.failedRecoverTaskInstanceFactory = failedRecoverTaskInstanceFactory;
}
public FailedRecoverTaskInstanceBuilder withTaskInstance(TaskInstance needRecoverTaskInstance) {
this.needRecoverTaskInstance = needRecoverTaskInstance;
return this;
}
@Override
public TaskInstance build() {
return failedRecoverTaskInstanceFactory.createTaskInstance(this);
}
}
}

70
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/PauseRecoverTaskInstanceFactory.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.engine.task.runnable;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import org.springframework.transaction.annotation.Transactional;
@Component
public class PauseRecoverTaskInstanceFactory
extends
AbstractTaskInstanceFactory<PauseRecoverTaskInstanceFactory.PauseRecoverTaskInstanceBuilder> {
@Autowired
private TaskInstanceDao taskInstanceDao;
@Override
public PauseRecoverTaskInstanceFactory.PauseRecoverTaskInstanceBuilder builder() {
return new PauseRecoverTaskInstanceBuilder(this);
}
@Transactional
@Override
public TaskInstance createTaskInstance(PauseRecoverTaskInstanceBuilder builder) {
final TaskInstance needRecoverTaskInstance = builder.needRecoverTaskInstance;
needRecoverTaskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
taskInstanceDao.updateById(needRecoverTaskInstance);
return needRecoverTaskInstance;
}
public static class PauseRecoverTaskInstanceBuilder implements ITaskInstanceFactory.ITaskInstanceBuilder {
private final PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory;
private TaskInstance needRecoverTaskInstance;
public PauseRecoverTaskInstanceBuilder(PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory) {
this.pauseRecoverTaskInstanceFactory = pauseRecoverTaskInstanceFactory;
}
public PauseRecoverTaskInstanceBuilder withTaskInstance(TaskInstance needRecoverTaskInstance) {
this.needRecoverTaskInstance = needRecoverTaskInstance;
return this;
}
@Override
public TaskInstance build() {
return pauseRecoverTaskInstanceFactory.createTaskInstance(this);
}
}
}

13
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java

@ -29,6 +29,12 @@ public class TaskInstanceFactories {
@Autowired @Autowired
private RetryTaskInstanceFactory retryTaskInstanceFactory; private RetryTaskInstanceFactory retryTaskInstanceFactory;
@Autowired
private FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory;
@Autowired
private PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory;
@Autowired @Autowired
private FailoverTaskInstanceFactory failoverTaskInstanceFactory; private FailoverTaskInstanceFactory failoverTaskInstanceFactory;
@ -40,8 +46,15 @@ public class TaskInstanceFactories {
return retryTaskInstanceFactory; return retryTaskInstanceFactory;
} }
public FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory() {
return failedRecoverTaskInstanceFactory;
}
public FailoverTaskInstanceFactory failoverTaskInstanceFactory() { public FailoverTaskInstanceFactory failoverTaskInstanceFactory() {
return failoverTaskInstanceFactory; return failoverTaskInstanceFactory;
} }
public PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory() {
return pauseRecoverTaskInstanceFactory;
}
} }

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverFailureTaskTrigger.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger; package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest;
@ -34,7 +35,14 @@ public class WorkflowInstanceRecoverFailureTaskTrigger
@Override @Override
protected WorkflowInstance constructWorkflowInstance(final WorkflowInstanceRecoverFailureTasksRequest workflowInstanceRecoverFailureTasksRequest) { protected WorkflowInstance constructWorkflowInstance(final WorkflowInstanceRecoverFailureTasksRequest workflowInstanceRecoverFailureTasksRequest) {
return getWorkflowInstance(workflowInstanceRecoverFailureTasksRequest.getWorkflowInstanceId()); WorkflowInstance workflowInstance =
getWorkflowInstance(workflowInstanceRecoverFailureTasksRequest.getWorkflowInstanceId());
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.SUBMITTED_SUCCESS,
CommandType.START_FAILURE_TASK_PROCESS.name());
workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1);
workflowInstance.setRestartTime(new Date());
workflowInstance.setEndTime(null);
return workflowInstance;
} }
@Override @Override

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverSuspendTaskTrigger.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger; package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest;
@ -34,7 +35,14 @@ public class WorkflowInstanceRecoverSuspendTaskTrigger
@Override @Override
protected WorkflowInstance constructWorkflowInstance(final WorkflowInstanceRecoverSuspendTasksRequest workflowInstanceRecoverSuspendTasksRequest) { protected WorkflowInstance constructWorkflowInstance(final WorkflowInstanceRecoverSuspendTasksRequest workflowInstanceRecoverSuspendTasksRequest) {
return getWorkflowInstance(workflowInstanceRecoverSuspendTasksRequest.getWorkflowInstanceId()); final WorkflowInstance workflowInstance =
getWorkflowInstance(workflowInstanceRecoverSuspendTasksRequest.getWorkflowInstanceId());
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.SUBMITTED_SUCCESS,
CommandType.START_FAILURE_TASK_PROCESS.name());
workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1);
workflowInstance.setRestartTime(new Date());
workflowInstance.setEndTime(null);
return workflowInstance;
} }
@Override @Override

7
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java

@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillReque
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse; import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutor; import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutor;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorHolder; import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorHolder;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorThreadPoolManager; import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorThreadPoolManager;
@ -54,16 +53,10 @@ public class LogicITaskInstanceKillOperationFunction
} }
try { try {
masterTaskExecutor.cancelTask(); masterTaskExecutor.cancelTask();
// todo: if we remove success then we don't need to cancel?
masterTaskExecutorThreadPool.removeMasterTaskExecutor(masterTaskExecutor);
return LogicTaskKillResponse.success(); return LogicTaskKillResponse.success();
} catch (MasterTaskExecuteException e) { } catch (MasterTaskExecuteException e) {
log.error("Cancel MasterTaskExecuteRunnable failed ", e); log.error("Cancel MasterTaskExecuteRunnable failed ", e);
return LogicTaskKillResponse.fail("Cancel MasterTaskExecuteRunnable failed: " + e.getMessage()); return LogicTaskKillResponse.fail("Cancel MasterTaskExecuteRunnable failed: " + e.getMessage());
} finally {
// todo: If cancel failed, we cannot remove the context?
MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskInstanceId);
MasterTaskExecutorHolder.removeMasterTaskExecutor(taskInstanceId);
} }
} finally { } finally {
LogUtils.removeTaskInstanceIdMDC(); LogUtils.removeTaskInstanceIdMDC();

6
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java

@ -95,6 +95,12 @@ public class AsyncMasterTaskDelayQueueLooper extends BaseDaemonThread implements
// put it back to the queue to get the status again. // put it back to the queue to get the status again.
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext); asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext);
break; break;
case PAUSE:
asyncTaskCallbackFunction.executePause();
break;
case KILL:
asyncTaskCallbackFunction.executeKilled();
break;
case SUCCESS: case SUCCESS:
asyncTaskCallbackFunction.executeSuccess(); asyncTaskCallbackFunction.executeSuccess();
break; break;

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java

@ -27,4 +27,7 @@ public interface AsyncTaskCallbackFunction {
void executeThrowing(Throwable throwable); void executeThrowing(Throwable throwable);
void executePause();
void executeKilled();
} }

14
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java

@ -51,6 +51,20 @@ public class AsyncTaskCallbackFunctionImpl implements AsyncTaskCallbackFunction
asyncMasterTaskExecuteRunnable.afterThrowing(throwable); asyncMasterTaskExecuteRunnable.afterThrowing(throwable);
} }
@Override
public void executePause() {
asyncMasterTaskExecuteRunnable.getTaskExecutionContext()
.setCurrentExecutionStatus(TaskExecutionStatus.PAUSE);
executeFinished();
}
@Override
public void executeKilled() {
asyncMasterTaskExecuteRunnable.getTaskExecutionContext()
.setCurrentExecutionStatus(TaskExecutionStatus.KILL);
executeFinished();
}
private void executeFinished() { private void executeFinished() {
TaskInstanceLogHeader.printFinalizeTaskHeader(); TaskInstanceLogHeader.printFinalizeTaskHeader();
int taskInstanceId = asyncMasterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId(); int taskInstanceId = asyncMasterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId();

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java

@ -33,6 +33,10 @@ public interface AsyncTaskExecuteFunction {
RUNNING, RUNNING,
PAUSE,
KILL,
SUCCESS, SUCCESS,
FAILED, FAILED,

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java

@ -17,9 +17,9 @@
package org.apache.dolphinscheduler.server.master.runner.execute; package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.dynamic.DynamicLogicTask; import org.apache.dolphinscheduler.server.master.runner.task.dynamic.DynamicLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask;
import java.util.Set; import java.util.Set;
@ -42,7 +42,7 @@ public class MasterTaskExecutorFactoryBuilder {
private static final Set<String> ASYNC_TASK_TYPE = Sets.newHashSet( private static final Set<String> ASYNC_TASK_TYPE = Sets.newHashSet(
DependentLogicTask.TASK_TYPE, DependentLogicTask.TASK_TYPE,
SubWorkflowLogicTask.TASK_TYPE, SubWorkflowLogicTaskChannelFactory.NAME,
DynamicLogicTask.TASK_TYPE); DynamicLogicTask.TASK_TYPE);
public MasterTaskExecutorFactory<? extends MasterTaskExecutor> createMasterTaskExecutorFactory(String taskType) { public MasterTaskExecutorFactory<? extends MasterTaskExecutor> createMasterTaskExecutorFactory(String taskType) {

11
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java

@ -20,8 +20,6 @@ package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters; import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
@ -37,15 +35,6 @@ public abstract class BaseAsyncLogicTask<T extends AbstractParameters> implement
log.info("Success initialize task parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters)); log.info("Success initialize task parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters));
} }
@Override
public void kill() {
MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId());
}
public void pause() throws MasterTaskExecuteException {
MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId());
}
@Override @Override
public TaskExecutionContext getTaskExecutionContext() { public TaskExecutionContext getTaskExecutionContext() {
return taskExecutionContext; return taskExecutionContext;

32
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java

@ -18,22 +18,18 @@
package org.apache.dolphinscheduler.server.master.runner.task.dependent; package org.apache.dolphinscheduler.server.master.runner.task.dependent;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.ProjectDao; import org.apache.dolphinscheduler.dao.repository.ProjectDao;
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao; import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
import java.util.Date;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.core.type.TypeReference;
@ -51,6 +47,8 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
private final IWorkflowExecutionRunnable workflowExecutionRunnable; private final IWorkflowExecutionRunnable workflowExecutionRunnable;
private DependentAsyncTaskExecuteFunction dependentAsyncTaskExecuteFunction;
public DependentLogicTask(TaskExecutionContext taskExecutionContext, public DependentLogicTask(TaskExecutionContext taskExecutionContext,
ProjectDao projectDao, ProjectDao projectDao,
WorkflowDefinitionDao workflowDefinitionDao, WorkflowDefinitionDao workflowDefinitionDao,
@ -72,34 +70,24 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
@Override @Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() { public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
return new DependentAsyncTaskExecuteFunction(taskExecutionContext, dependentAsyncTaskExecuteFunction = new DependentAsyncTaskExecuteFunction(taskExecutionContext,
taskParameters, taskParameters,
projectDao, projectDao,
workflowDefinitionDao, workflowDefinitionDao,
taskDefinitionDao, taskDefinitionDao,
taskInstanceDao, taskInstanceDao,
workflowInstanceDao); workflowInstanceDao);
return dependentAsyncTaskExecuteFunction;
} }
@Override @Override
public void pause() throws MasterTaskExecuteException { public void pause() throws MasterTaskExecuteException {
if (workflowExecutionRunnable == null) { // todo: support pause
log.error("Cannot find the WorkflowExecuteRunnable"); }
return;
} @Override
TaskInstance taskInstance = workflowExecutionRunnable public void kill() throws MasterTaskExecuteException {
.getWorkflowExecuteContext() // todo: support kill
.getWorkflowExecutionGraph()
.getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId())
.getTaskInstance();
if (taskInstance == null) {
log.error("Cannot find the TaskInstance in workflowExecuteRunnable");
return;
}
taskInstance.setState(TaskExecutionStatus.PAUSE);
taskInstance.setEndTime(new Date());
taskInstanceDao.upsertTaskInstance(taskInstance);
super.pause();
} }
} }

5
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java

@ -250,6 +250,11 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
return dynamicInputParameters; return dynamicInputParameters;
} }
@Override
public void pause() throws MasterTaskExecuteException {
// todo: support pause
}
@Override @Override
public void kill() { public void kill() {
try { try {

45
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java

@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import java.time.Duration; import java.time.Duration;
@ -32,42 +31,36 @@ public class SubWorkflowAsyncTaskExecuteFunction implements AsyncTaskExecuteFunc
private static final Duration SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL = Duration.ofSeconds(10); private static final Duration SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL = Duration.ofSeconds(10);
private final TaskExecutionContext taskExecutionContext;
private final WorkflowInstanceDao workflowInstanceDao; private final WorkflowInstanceDao workflowInstanceDao;
private WorkflowInstance subWorkflowInstance;
private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext; private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
public SubWorkflowAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext, public SubWorkflowAsyncTaskExecuteFunction(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext,
WorkflowInstanceDao workflowInstanceDao) { final WorkflowInstanceDao workflowInstanceDao) {
this.taskExecutionContext = taskExecutionContext; this.subWorkflowLogicTaskRuntimeContext = subWorkflowLogicTaskRuntimeContext;
this.workflowInstanceDao = workflowInstanceDao; this.workflowInstanceDao = workflowInstanceDao;
} }
@Override @Override
public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() { public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId);
if (subWorkflowInstance == null) { if (subWorkflowInstance == null) {
createSubWorkflowInstanceIfAbsent(); log.info("Cannot find the SubWorkflow instance: {}, maybe it has been deleted", subWorkflowInstanceId);
return AsyncTaskExecutionStatus.FAILED;
} }
if (subWorkflowInstance == null) { switch (subWorkflowInstance.getState()) {
log.info("The sub workflow instance doesn't created"); case PAUSE:
return AsyncTaskExecutionStatus.RUNNING; return AsyncTaskExecutionStatus.PAUSE;
} case STOP:
subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstance.getId()); return AsyncTaskExecutionStatus.KILL;
if (subWorkflowInstance != null && subWorkflowInstance.getState().isFinished()) { case SUCCESS:
return subWorkflowInstance.getState().isSuccess() ? AsyncTaskExecutionStatus.SUCCESS return AsyncTaskExecutionStatus.SUCCESS;
: AsyncTaskExecutionStatus.FAILED; case FAILURE:
return AsyncTaskExecutionStatus.FAILED;
default:
return AsyncTaskExecutionStatus.RUNNING;
} }
return AsyncTaskExecutionStatus.RUNNING;
}
private void createSubWorkflowInstanceIfAbsent() {
// todo: we need to creat sub workflow instance here, rather than create command
// create command might occur duplicate sub workflow instance when failover
// generate the sub workflow instance
subWorkflowInstance = workflowInstanceDao.querySubWorkflowInstanceByParentId(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
} }
@Override @Override

107
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java

@ -0,0 +1,107 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverFailureTaskTrigger;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverSuspendTaskTrigger;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.trigger.SubWorkflowManualTrigger;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class SubWorkflowControlClient {
@Autowired
private WorkflowInstanceDao workflowInstanceDao;
@Autowired
private SubWorkflowManualTrigger subWorkflowManualTrigger;
@Autowired
private WorkflowInstanceRecoverFailureTaskTrigger workflowInstanceRecoverFailureTaskTrigger;
@Autowired
private WorkflowInstanceRecoverSuspendTaskTrigger workflowInstanceRecoverSuspendTaskTrigger;
public Integer triggerSubWorkflow(final WorkflowManualTriggerRequest workflowManualTriggerRequest) {
return subWorkflowManualTrigger.triggerWorkflow(workflowManualTriggerRequest).getWorkflowInstanceId();
}
public WorkflowInstanceRecoverFailureTasksResponse triggerFromFailureTasks(
final WorkflowInstanceRecoverFailureTasksRequest recoverFailureTasksRequest) {
return workflowInstanceRecoverFailureTaskTrigger.triggerWorkflow(recoverFailureTasksRequest);
}
public WorkflowInstanceRecoverSuspendTasksResponse triggerFromSuspendTasks(
final WorkflowInstanceRecoverSuspendTasksRequest recoverSuspendTasksRequest) {
return workflowInstanceRecoverSuspendTaskTrigger.triggerWorkflow(recoverSuspendTasksRequest);
}
public WorkflowInstancePauseResponse pauseWorkflowInstance(
final WorkflowInstancePauseRequest workflowInstancePauseRequest) throws MasterTaskExecuteException {
final Integer subWorkflowInstanceId = workflowInstancePauseRequest.getWorkflowInstanceId();
final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId);
if (subWorkflowInstance.getState() != WorkflowExecutionStatus.RUNNING_EXECUTION) {
return WorkflowInstancePauseResponse.fail("SubWorkflow instance is not running, cannot pause");
}
try {
return Clients
.withService(IWorkflowControlClient.class)
.withHost(subWorkflowInstance.getHost())
.pauseWorkflowInstance(new WorkflowInstancePauseRequest(subWorkflowInstanceId));
} catch (Exception e) {
throw new MasterTaskExecuteException("Pause SubWorkflow: " + subWorkflowInstance.getName() + " failed", e);
}
}
public WorkflowInstanceStopResponse stopWorkflowInstance(
final WorkflowInstanceStopRequest workflowInstanceStopRequest) throws MasterTaskExecuteException {
final Integer subWorkflowInstanceId = workflowInstanceStopRequest.getWorkflowInstanceId();
final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId);
if (subWorkflowInstance.getState() != WorkflowExecutionStatus.RUNNING_EXECUTION) {
return WorkflowInstanceStopResponse.fail("SubWorkflow instance is not running, cannot stop");
}
try {
return Clients
.withService(IWorkflowControlClient.class)
.withHost(subWorkflowInstance.getHost())
.stopWorkflowInstance(new WorkflowInstanceStopRequest(subWorkflowInstance.getId()));
} catch (Exception e) {
throw new MasterTaskExecuteException("Kill SubWorkflow: " + subWorkflowInstance.getName() + " failed", e);
}
}
}

258
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java

@ -17,36 +17,30 @@
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
import org.apache.dolphinscheduler.common.enums.TaskDependType; import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition; import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.dao.entity.WorkflowInstanceRelation;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao; import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.Clients; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceMapDao;
import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.command.ICommandParam; import org.apache.dolphinscheduler.extract.master.command.ICommandParam;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters; import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
import org.apache.commons.lang3.StringUtils;
import java.util.ArrayList;
import java.util.Date;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContext;
@ -56,19 +50,11 @@ import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j @Slf4j
public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameters> { public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameters> {
public static final String TASK_TYPE = "SUB_PROCESS"; private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager;
private final WorkflowInstanceDao workflowInstanceDao;
private final WorkflowDefinitionDao workflowDefinitionDao;
private final CommandDao commandDao;
private final IWorkflowExecutionRunnable workflowExecutionRunnable; private final IWorkflowExecutionRunnable workflowExecutionRunnable;
private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext; private final ApplicationContext applicationContext;
public SubWorkflowLogicTask(final TaskExecutionContext taskExecutionContext, public SubWorkflowLogicTask(final TaskExecutionContext taskExecutionContext,
final IWorkflowExecutionRunnable workflowExecutionRunnable, final IWorkflowExecutionRunnable workflowExecutionRunnable,
@ -76,141 +62,165 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameter
super(taskExecutionContext, super(taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SubProcessParameters>() { JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SubProcessParameters>() {
})); }));
this.workflowDefinitionDao = applicationContext.getBean(WorkflowDefinitionDao.class);
this.workflowInstanceDao = applicationContext.getBean(WorkflowInstanceDao.class);
this.commandDao = applicationContext.getBean(CommandDao.class);
this.logicTaskInstanceExecutionEventSenderManager =
applicationContext.getBean(LogicTaskInstanceExecutionEventSenderManager.class);
this.workflowExecutionRunnable = workflowExecutionRunnable; this.workflowExecutionRunnable = workflowExecutionRunnable;
this.subWorkflowLogicTaskRuntimeContext = initializeSubWorkflowLogicTaskRuntimeContext(); this.applicationContext = applicationContext;
taskExecutionContext.setAppIds(JSONUtils.toJsonString(subWorkflowLogicTaskRuntimeContext)); this.subWorkflowLogicTaskRuntimeContext = JSONUtils.parseObject(
logicTaskInstanceExecutionEventSenderManager.runningEventSender().sendMessage(taskExecutionContext); taskExecutionContext.getAppIds(),
SubWorkflowLogicTaskRuntimeContext.class);
} }
@Override @Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() { public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
return new SubWorkflowAsyncTaskExecuteFunction(taskExecutionContext, workflowInstanceDao); subWorkflowLogicTaskRuntimeContext = initializeSubWorkflowInstance();
upsertSubWorkflowRelation();
taskExecutionContext.setAppIds(JSONUtils.toJsonString(subWorkflowLogicTaskRuntimeContext));
applicationContext
.getBean(LogicTaskInstanceExecutionEventSenderManager.class)
.runningEventSender()
.sendMessage(taskExecutionContext);
return new SubWorkflowAsyncTaskExecuteFunction(
subWorkflowLogicTaskRuntimeContext,
applicationContext.getBean(WorkflowInstanceDao.class));
} }
@Override @Override
public void pause() throws MasterTaskExecuteException { public void pause() throws MasterTaskExecuteException {
WorkflowInstance subWorkflowInstance = if (subWorkflowLogicTaskRuntimeContext == null) {
workflowInstanceDao.querySubWorkflowInstanceByParentId(taskExecutionContext.getProcessInstanceId(), log.info("subWorkflowLogicTaskRuntimeContext is null cannot pause");
taskExecutionContext.getTaskInstanceId()); return;
}
try { final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
WorkflowInstancePauseResponse pauseResponse = Clients final WorkflowInstancePauseResponse pauseResponse = applicationContext
.withService(IWorkflowControlClient.class) .getBean(SubWorkflowControlClient.class)
.withHost(subWorkflowInstance.getHost()) .pauseWorkflowInstance(new WorkflowInstancePauseRequest(subWorkflowInstanceId));
.pauseWorkflowInstance(new WorkflowInstancePauseRequest(subWorkflowInstance.getId())); if (pauseResponse.isSuccess()) {
if (pauseResponse.isSuccess()) { log.info("Pause sub workflowInstance: id={}", subWorkflowInstanceId + " success");
log.info("Pause sub workflowInstance: {}", subWorkflowInstance.getName() + " success"); } else {
} else { log.info("Pause sub workflowInstance: id={} failed with response: {}", subWorkflowInstanceId,
throw new MasterTaskExecuteException( pauseResponse);
"Pause sub workflowInstance: " + subWorkflowInstance.getName() + " failed with response: "
+ pauseResponse);
}
} catch (MasterTaskExecuteException me) {
throw me;
} catch (Exception e) {
throw new MasterTaskExecuteException(
"Send pause request to SubWorkflow's master: " + subWorkflowInstance.getName() + " failed", e);
} }
} }
@Override @Override
public void kill() { public void kill() throws MasterTaskExecuteException {
WorkflowInstance subWorkflowInstance = if (subWorkflowLogicTaskRuntimeContext == null) {
workflowInstanceDao.querySubWorkflowInstanceByParentId(taskExecutionContext.getProcessInstanceId(), log.info("subWorkflowLogicTaskRuntimeContext is null cannot kill");
taskExecutionContext.getTaskInstanceId());
if (subWorkflowInstance == null) {
log.info("SubWorkflow instance is null");
return; return;
} }
try { final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
WorkflowInstanceStopResponse stopResponse = Clients final WorkflowInstanceStopResponse stopResponse = applicationContext
.withService(IWorkflowControlClient.class) .getBean(SubWorkflowControlClient.class)
.withHost(subWorkflowInstance.getHost()) .stopWorkflowInstance(new WorkflowInstanceStopRequest(subWorkflowInstanceId));
.stopWorkflowInstance(new WorkflowInstanceStopRequest(subWorkflowInstance.getId())); if (stopResponse.isSuccess()) {
if (stopResponse.isSuccess()) { log.info("Kill sub workflowInstance: id={}", subWorkflowInstanceId + " success");
log.info("Kill sub workflowInstance: {}", subWorkflowInstance.getName() + " success"); } else {
} else { log.info("Kill sub workflowInstance: id={} failed with response: {}", subWorkflowInstanceId, stopResponse);
log.error("Kill sub workflowInstance: {} failed with response: {}", subWorkflowInstance.getName(),
stopResponse);
}
} catch (Exception e) {
log.error("Send kill request to SubWorkflow's master: {} failed", subWorkflowInstance.getHost(), e);
} }
} }
private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowLogicTaskRuntimeContext() { private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowInstance() {
if (taskExecutionContext.isFailover() && StringUtils.isNotEmpty(taskExecutionContext.getAppIds())) { // todo: doFailover if the runtime context is not null and task is generated by failover
return JSONUtils.parseObject(taskExecutionContext.getAppIds(), SubWorkflowLogicTaskRuntimeContext.class);
if (subWorkflowLogicTaskRuntimeContext == null) {
return triggerNewSubWorkflow();
} }
// If the task is not in failover mode or the runtime context is not exist
// then we should create the runtime context by command type we should start/recover from failure... switch (workflowExecutionRunnable.getWorkflowInstance().getCommandType()) {
final IWorkflowExecuteContext workflowExecuteContext = workflowExecutionRunnable.getWorkflowExecuteContext();
final WorkflowInstance workflowInstance = workflowExecuteContext.getWorkflowInstance();
switch (workflowInstance.getCommandType()) {
case START_PROCESS:
case SCHEDULER:
case START_CURRENT_TASK_PROCESS:
case RECOVER_SERIAL_WAIT:
case COMPLEMENT_DATA:
return createSubWorkflowInstanceFromWorkflowDefinition();
case REPEAT_RUNNING:
case START_FAILURE_TASK_PROCESS:
case RECOVER_SUSPENDED_PROCESS: case RECOVER_SUSPENDED_PROCESS:
return createSubWorkflowInstanceWithWorkflowInstance(); return recoverFromSuspendTasks();
case START_FAILURE_TASK_PROCESS:
return recoverFromFailedTasks();
default: default:
throw new IllegalArgumentException("Unsupported command type: " + workflowInstance.getCommandType()); return triggerNewSubWorkflow();
}
}
private SubWorkflowLogicTaskRuntimeContext recoverFromFailedTasks() {
final SubWorkflowControlClient subWorkflowControlClient =
applicationContext.getBean(SubWorkflowControlClient.class);
if (subWorkflowLogicTaskRuntimeContext == null) {
log.info("The task: {} triggerType is FAILED_RECOVER but runtimeContext is null will trigger again",
taskExecutionContext.getTaskName());
return triggerNewSubWorkflow();
} }
final WorkflowInstanceRecoverFailureTasksRequest recoverFailureTasksRequest =
WorkflowInstanceRecoverFailureTasksRequest.builder()
.workflowInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId())
.userId(taskExecutionContext.getExecutorId())
.build();
subWorkflowControlClient.triggerFromFailureTasks(recoverFailureTasksRequest);
return subWorkflowLogicTaskRuntimeContext;
} }
private SubWorkflowLogicTaskRuntimeContext createSubWorkflowInstanceFromWorkflowDefinition() { private SubWorkflowLogicTaskRuntimeContext recoverFromSuspendTasks() {
final SubWorkflowControlClient subWorkflowControlClient =
applicationContext.getBean(SubWorkflowControlClient.class);
if (subWorkflowLogicTaskRuntimeContext == null) {
log.info("The task: {} is recover from suspend but runtimeContext is null will trigger again",
taskExecutionContext.getTaskName());
return triggerNewSubWorkflow();
}
final WorkflowInstanceRecoverSuspendTasksRequest recoverSuspendTasksRequest =
WorkflowInstanceRecoverSuspendTasksRequest.builder()
.workflowInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId())
.userId(taskExecutionContext.getExecutorId())
.build();
subWorkflowControlClient.triggerFromSuspendTasks(recoverSuspendTasksRequest);
return subWorkflowLogicTaskRuntimeContext;
}
private SubWorkflowLogicTaskRuntimeContext triggerNewSubWorkflow() {
final WorkflowInstance workflowInstance = workflowExecutionRunnable.getWorkflowInstance(); final WorkflowInstance workflowInstance = workflowExecutionRunnable.getWorkflowInstance();
final WorkflowDefinition subWorkflowDefinition = applicationContext.getBean(WorkflowDefinitionDao.class)
.queryByCode(taskParameters.getProcessDefinitionCode())
.orElseThrow(() -> new IllegalArgumentException(
"Cannot find the sub workflow definition: " + taskParameters.getProcessDefinitionCode()));
final ICommandParam commandParam = final ICommandParam commandParam =
JSONUtils.parseObject(workflowInstance.getCommandParam(), ICommandParam.class); JSONUtils.parseObject(workflowInstance.getCommandParam(), ICommandParam.class);
final RunWorkflowCommandParam runWorkflowCommandParam =
RunWorkflowCommandParam.builder()
.commandParams(new ArrayList<>(taskExecutionContext.getPrepareParamsMap().values()))
.startNodes(new ArrayList<>())
.timeZone(commandParam.getTimeZone())
.subWorkflowInstance(true)
.build();
final WorkflowDefinition subWorkflowDefinition = getSubWorkflowDefinition(); final WorkflowManualTriggerRequest workflowManualTriggerRequest = WorkflowManualTriggerRequest.builder()
final Command command = Command.builder() .userId(taskExecutionContext.getExecutorId())
.commandType(workflowInstance.getCommandType()) .workflowDefinitionCode(subWorkflowDefinition.getCode())
.processDefinitionCode(subWorkflowDefinition.getCode()) .workflowDefinitionVersion(subWorkflowDefinition.getVersion())
.processDefinitionVersion(subWorkflowDefinition.getVersion())
.executorId(workflowInstance.getExecutorId())
.commandParam(JSONUtils.toJsonString(runWorkflowCommandParam))
.taskDependType(TaskDependType.TASK_POST)
.failureStrategy(workflowInstance.getFailureStrategy()) .failureStrategy(workflowInstance.getFailureStrategy())
.warningType(workflowInstance.getWarningType()) .warningType(workflowInstance.getWarningType())
.warningGroupId(workflowInstance.getWarningGroupId()) .warningGroupId(workflowInstance.getWarningGroupId())
.startTime(new Date()) .workflowInstancePriority(workflowInstance.getProcessInstancePriority())
.processInstancePriority(workflowInstance.getProcessInstancePriority()) .workerGroup(workflowInstance.getWorkerGroup())
.updateTime(new Date()) .tenantCode(workflowInstance.getTenantCode())
.workerGroup(taskExecutionContext.getWorkerGroup()) .environmentCode(workflowInstance.getEnvironmentCode())
.tenantCode(taskExecutionContext.getTenantCode()) // todo: transport varpool and local params
.dryRun(taskExecutionContext.getDryRun()) .startParamList(commandParam.getCommandParams())
.testFlag(taskExecutionContext.getTestFlag()) .dryRun(Flag.of(workflowInstance.getDryRun()))
.build(); .testFlag(Flag.of(workflowInstance.getTestFlag()))
commandDao.insert(command);
return SubWorkflowLogicTaskRuntimeContext.builder()
.subWorkflowCommandId(command.getId())
.build(); .build();
final Integer subWorkflowInstanceId = applicationContext
.getBean(SubWorkflowControlClient.class)
.triggerSubWorkflow(workflowManualTriggerRequest);
return SubWorkflowLogicTaskRuntimeContext.of(subWorkflowInstanceId);
} }
private SubWorkflowLogicTaskRuntimeContext createSubWorkflowInstanceWithWorkflowInstance() { private void upsertSubWorkflowRelation() {
return null; final WorkflowInstanceMapDao workflowInstanceMapDao = applicationContext.getBean(WorkflowInstanceMapDao.class);
} WorkflowInstanceRelation workflowInstanceRelation = workflowInstanceMapDao.queryWorkflowMapByParent(
taskExecutionContext.getProcessInstanceId(),
private WorkflowDefinition getSubWorkflowDefinition() { taskExecutionContext.getTaskInstanceId());
return workflowDefinitionDao.queryByCode(taskParameters.getProcessDefinitionCode()).orElseThrow( if (workflowInstanceRelation == null) {
() -> new IllegalArgumentException( workflowInstanceRelation = WorkflowInstanceRelation.builder()
"Cannot find the sub workflow definition: " + taskParameters.getProcessDefinitionCode())); .parentProcessInstanceId(taskExecutionContext.getProcessInstanceId())
.parentTaskInstanceId(taskExecutionContext.getTaskInstanceId())
.processInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId())
.build();
workflowInstanceMapDao.insert(workflowInstanceRelation);
} else {
workflowInstanceRelation
.setProcessInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId());
workflowInstanceMapDao.updateById(workflowInstanceRelation);
}
} }
} }

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
@ -52,6 +53,6 @@ public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactor
@Override @Override
public String getTaskType() { public String getTaskType() {
return SubWorkflowLogicTask.TASK_TYPE; return SubWorkflowLogicTaskChannelFactory.NAME;
} }
} }

11
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java

@ -17,8 +17,6 @@
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Builder; import lombok.Builder;
import lombok.Data; import lombok.Data;
@ -33,9 +31,12 @@ import lombok.NoArgsConstructor;
@NoArgsConstructor @NoArgsConstructor
public class SubWorkflowLogicTaskRuntimeContext { public class SubWorkflowLogicTaskRuntimeContext {
private Integer subWorkflowCommandId;
private Integer subWorkflowInstanceId; private Integer subWorkflowInstanceId;
private WorkflowExecutionStatus subWorkflowExecutionStatus; public static SubWorkflowLogicTaskRuntimeContext of(final Integer subWorkflowInstanceId) {
return SubWorkflowLogicTaskRuntimeContext.builder()
.subWorkflowInstanceId(subWorkflowInstanceId)
.build();
}
} }

40
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.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.runner.task.subworkflow.trigger;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowManualTrigger;
import org.springframework.stereotype.Component;
/**
* Manual trigger of the workflow, used to trigger the workflow and generate the workflow instance in the manual way.
*/
@Component
public class SubWorkflowManualTrigger extends WorkflowManualTrigger {
@Override
protected WorkflowInstance constructWorkflowInstance(final WorkflowManualTriggerRequest workflowManualTriggerRequest) {
final WorkflowInstance workflowInstance = super.constructWorkflowInstance(workflowManualTriggerRequest);
workflowInstance.setIsSubProcess(Flag.YES);
return workflowInstance;
}
}

2
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java

@ -37,7 +37,7 @@ public class WorkflowITContext {
private Project project; private Project project;
private WorkflowDefinition workflow; private List<WorkflowDefinition> workflows;
private WorkflowInstance workflowInstance; private WorkflowInstance workflowInstance;

16
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java

@ -77,7 +77,7 @@ public class WorkflowITContextFactory {
public WorkflowITContext initializeContextFromYaml(final String yamlPath) { public WorkflowITContext initializeContextFromYaml(final String yamlPath) {
final WorkflowITContext workflowITContext = YamlFactory.load(yamlPath); final WorkflowITContext workflowITContext = YamlFactory.load(yamlPath);
initializeProjectToDB(workflowITContext.getProject()); initializeProjectToDB(workflowITContext.getProject());
initializeWorkflowDefinitionToDB(workflowITContext.getWorkflow()); initializeWorkflowDefinitionToDB(workflowITContext.getWorkflows());
initializeTaskDefinitionsToDB(workflowITContext.getTasks()); initializeTaskDefinitionsToDB(workflowITContext.getTasks());
initializeTaskRelationsToDB(workflowITContext.getTaskRelations()); initializeTaskRelationsToDB(workflowITContext.getTaskRelations());
if (workflowITContext.getWorkflowInstance() != null) { if (workflowITContext.getWorkflowInstance() != null) {
@ -99,12 +99,14 @@ public class WorkflowITContextFactory {
workflowInstanceDao.insert(workflowInstance); workflowInstanceDao.insert(workflowInstance);
} }
private void initializeWorkflowDefinitionToDB(final WorkflowDefinition workflowDefinition) { private void initializeWorkflowDefinitionToDB(final List<WorkflowDefinition> workflowDefinitions) {
workflowDefinitionDao.insert(workflowDefinition); for (final WorkflowDefinition workflowDefinition : workflowDefinitions) {
final WorkflowDefinitionLog workflowDefinitionLog = new WorkflowDefinitionLog(workflowDefinition); workflowDefinitionDao.insert(workflowDefinition);
workflowDefinitionLog.setOperator(workflowDefinition.getUserId()); final WorkflowDefinitionLog workflowDefinitionLog = new WorkflowDefinitionLog(workflowDefinition);
workflowDefinitionLog.setOperateTime(new Date()); workflowDefinitionLog.setOperator(workflowDefinition.getUserId());
workflowDefinitionLogDao.insert(workflowDefinitionLog); workflowDefinitionLog.setOperateTime(new Date());
workflowDefinitionLogDao.insert(workflowDefinitionLog);
}
} }
private void initializeTaskDefinitionsToDB(final List<TaskDefinition> taskDefinitions) { private void initializeTaskDefinitionsToDB(final List<TaskDefinition> taskDefinitions) {

11
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java

@ -28,6 +28,8 @@ import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowB
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRepeatRunningRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRepeatRunningRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
@ -114,6 +116,15 @@ public class WorkflowOperator {
return workflowInstanceController.pauseWorkflowInstance(workflowInstancePauseRequest); return workflowInstanceController.pauseWorkflowInstance(workflowInstancePauseRequest);
} }
public WorkflowInstanceRecoverSuspendTasksResponse recoverSuspendWorkflowInstance(Integer workflowInstanceId) {
final WorkflowInstanceRecoverSuspendTasksRequest workflowInstancePauseRequest =
WorkflowInstanceRecoverSuspendTasksRequest.builder()
.workflowInstanceId(workflowInstanceId)
.userId(1)
.build();
return workflowInstanceController.triggerFromSuspendTasks(workflowInstancePauseRequest);
}
public WorkflowInstanceStopResponse stopWorkflowInstance(Integer workflowInstanceId) { public WorkflowInstanceStopResponse stopWorkflowInstance(Integer workflowInstanceId) {
final WorkflowInstanceStopRequest workflowInstanceStopRequest = final WorkflowInstanceStopRequest workflowInstanceStopRequest =
new WorkflowInstanceStopRequest(workflowInstanceId); new WorkflowInstanceStopRequest(workflowInstanceId);

2
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java

@ -62,7 +62,7 @@ public class WorkflowBackfillIT extends AbstractMasterIntegrationTest {
public void testSerialBackfillWorkflow_with_oneSuccessTask() { public void testSerialBackfillWorkflow_with_oneSuccessTask() {
final String yaml = "/it/backfill/workflow_with_one_fake_task_success.yaml"; final String yaml = "/it/backfill/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder() final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder()
.backfillTimeList( .backfillTimeList(

30
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java

@ -56,7 +56,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withSubmittedTasks() { public void testGlobalFailover_runningWorkflow_withSubmittedTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml"; final String yaml = "/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -92,7 +92,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withDispatchTasks() { public void testGlobalFailover_runningWorkflow_withDispatchTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml"; final String yaml = "/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -132,7 +132,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withRunningTasks() { public void testGlobalFailover_runningWorkflow_withRunningTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_running_fake_task.yaml"; final String yaml = "/it/failover/running_workflowInstance_with_one_running_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -172,7 +172,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withSuccessTasks() { public void testGlobalFailover_runningWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_success_fake_task.yaml"; final String yaml = "/it/failover/running_workflowInstance_with_one_success_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -206,7 +206,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withFailedTasks() { public void testGlobalFailover_runningWorkflow_withFailedTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml"; final String yaml = "/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -241,7 +241,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withSubmittedTasks() { public void testGlobalFailover_readyPauseWorkflow_withSubmittedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml"; final String yaml = "/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -276,7 +276,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withDispatchedTasks() { public void testGlobalFailover_readyPauseWorkflow_withDispatchedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml"; final String yaml = "/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -316,7 +316,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withSuccessTasks() { public void testGlobalFailover_readyPauseWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml"; final String yaml = "/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -351,7 +351,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withFailedTasks() { public void testGlobalFailover_readyPauseWorkflow_withFailedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml"; final String yaml = "/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -386,7 +386,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withPausedTasks() { public void testGlobalFailover_readyPauseWorkflow_withPausedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml"; final String yaml = "/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -421,7 +421,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withSubmittedTasks() { public void testGlobalFailover_readyStopWorkflow_withSubmittedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml"; final String yaml = "/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -456,7 +456,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withDispatchedTasks() { public void testGlobalFailover_readyStopWorkflow_withDispatchedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml"; final String yaml = "/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -496,7 +496,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withSuccessTasks() { public void testGlobalFailover_readyStopWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml"; final String yaml = "/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -531,7 +531,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withFailedTasks() { public void testGlobalFailover_readyStopWorkflow_withFailedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml"; final String yaml = "/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -566,7 +566,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withKilledTasks() { public void testGlobalFailover_readyStopWorkflow_withKilledTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml"; final String yaml = "/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date())); systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));

87
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java

@ -20,7 +20,9 @@ package org.apache.dolphinscheduler.server.master.it.cases;
import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await; import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition; import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@ -32,6 +34,7 @@ import org.apache.dolphinscheduler.server.master.it.WorkflowITContextFactory;
import org.apache.dolphinscheduler.server.master.it.WorkflowOperator; import org.apache.dolphinscheduler.server.master.it.WorkflowOperator;
import java.time.Duration; import java.time.Duration;
import java.util.List;
import org.assertj.core.api.Assertions; import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.DisplayName;
@ -60,7 +63,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_oneSuccessTask() { public void testPauseWorkflow_with_oneSuccessTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_success.yaml"; final String yaml = "/it/pause/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -111,7 +114,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_oneFailedTask() { public void testPauseWorkflow_with_oneFailedTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_failed.yaml"; final String yaml = "/it/pause/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -162,7 +165,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_threeParallelSuccessTask() { public void testPauseWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml"; final String yaml = "/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -236,4 +239,82 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
assertThat(workflowRepository.getAll()).isEmpty(); assertThat(workflowRepository.getAll()).isEmpty();
} }
@Test
@DisplayName("Test pause a workflow with one sub workflow task")
public void testPauseWorkflow_with_subWorkflowTask_success() {
final String yaml = "/it/pause/workflow_with_sub_workflow_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryWorkflowInstance(context.getWorkflows().get(1)))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
});
assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess());
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.PAUSE);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.NO);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("sub_logic_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.PAUSE);
});
final WorkflowDefinition subWorkflowDefinition = context.getWorkflows().get(1);
Assertions
.assertThat(repository.queryWorkflowInstance(subWorkflowDefinition))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.PAUSE);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.YES);
});
final List<TaskInstance> taskInstances = repository.queryTaskInstance(subWorkflowDefinition);
Assertions
.assertThat(taskInstances)
.hasSize(2);
Assertions
.assertThat(taskInstances.get(0).getState())
.isEqualTo(TaskExecutionStatus.SUCCESS);
Assertions
.assertThat(taskInstances.get(1).getState())
.isEqualTo(TaskExecutionStatus.PAUSE);
});
assertThat(workflowRepository.getAll()).isEmpty();
}
} }

136
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverPauseIT.java

@ -0,0 +1,136 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.it.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.it.Repository;
import org.apache.dolphinscheduler.server.master.it.WorkflowITContext;
import org.apache.dolphinscheduler.server.master.it.WorkflowITContextFactory;
import org.apache.dolphinscheduler.server.master.it.WorkflowOperator;
import java.time.Duration;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for pausing a workflow instance.
*/
public class WorkflowInstanceRecoverPauseIT extends AbstractMasterIntegrationTest {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@Autowired
private IWorkflowRepository workflowRepository;
@Autowired
private Repository repository;
@Test
@DisplayName("Test recover a a workflow which is paused with one sub workflow task")
public void testRecoverPausedWorkflow_with_subWorkflowTask_success() {
final String yaml = "/it/recover_paused/workflow_with_sub_workflow_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(context.getWorkflows().get(1)))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
});
assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess());
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.PAUSE);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.NO);
});
});
assertThat(workflowOperator.recoverSuspendWorkflowInstance(workflowInstanceId).isSuccess());
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.matches(
workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.SUCCESS);
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
});
List<WorkflowInstance> subWorkflowInstances =
repository.queryWorkflowInstance(context.getWorkflows().get(1));
Assertions
.assertThat(subWorkflowInstances)
.satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.SUCCESS));
Assertions
.assertThat(repository.queryTaskInstance(subWorkflowInstances.get(0).getId()))
.hasSize(2)
.satisfies(taskInstance -> {
assertThat(taskInstance.get(0).getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
assertThat(taskInstance.get(0).getFlag()).isEqualTo(Flag.YES);
assertThat(taskInstance.get(1).getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
assertThat(taskInstance.get(1).getFlag()).isEqualTo(Flag.YES);
});
});
assertThat(workflowRepository.getAll()).isEmpty();
}
}

147
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverStopIT.java

@ -0,0 +1,147 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.it.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.it.Repository;
import org.apache.dolphinscheduler.server.master.it.WorkflowITContext;
import org.apache.dolphinscheduler.server.master.it.WorkflowITContextFactory;
import org.apache.dolphinscheduler.server.master.it.WorkflowOperator;
import java.time.Duration;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for pausing a workflow instance.
*/
public class WorkflowInstanceRecoverStopIT extends AbstractMasterIntegrationTest {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@Autowired
private IWorkflowRepository workflowRepository;
@Autowired
private Repository repository;
@Test
@DisplayName("Test recover a workflow which is stopped with one sub workflow task")
public void testRecoverStoppedWorkflow_with_subWorkflowTask_success() {
final String yaml = "/it/recover_stopped/workflow_with_sub_workflow_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(context.getWorkflows().get(1)))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
});
assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess());
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.NO);
});
});
assertThat(workflowOperator.recoverSuspendWorkflowInstance(workflowInstanceId).isSuccess());
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.matches(
workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.SUCCESS);
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.hasSize(2)
.satisfies(taskInstance -> {
assertThat(taskInstance.get(0).getState()).isEqualTo(TaskExecutionStatus.KILL);
assertThat(taskInstance.get(0).getFlag()).isEqualTo(Flag.NO);
assertThat(taskInstance.get(1).getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
assertThat(taskInstance.get(1).getFlag()).isEqualTo(Flag.YES);
});
List<WorkflowInstance> subWorkflowInstances =
repository.queryWorkflowInstance(context.getWorkflows().get(1));
Assertions
.assertThat(subWorkflowInstances)
.satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.SUCCESS));
Assertions
.assertThat(repository.queryTaskInstance(subWorkflowInstances.get(0).getId()))
.hasSize(3)
.satisfies(taskInstance -> {
assertThat(taskInstance.get(0).getState()).isEqualTo(TaskExecutionStatus.KILL);
assertThat(taskInstance.get(0).getFlag()).isEqualTo(Flag.NO);
assertThat(taskInstance.get(1).getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
assertThat(taskInstance.get(1).getFlag()).isEqualTo(Flag.YES);
assertThat(taskInstance.get(2).getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
assertThat(taskInstance.get(1).getFlag()).isEqualTo(Flag.YES);
});
});
assertThat(workflowRepository.getAll()).isEmpty();
}
}

79
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java

@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.server.master.it.cases;
import static com.google.common.truth.Truth.assertThat; import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await; import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition; import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
@ -62,7 +63,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_oneSuccessTask() { public void testStopWorkflow_with_oneSuccessTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_success.yaml"; final String yaml = "/it/stop/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -111,7 +112,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_oneFailedTask() { public void testStopWorkflow_with_oneFailedTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_failed.yaml"; final String yaml = "/it/stop/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -160,7 +161,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_threeParallelSuccessTask() { public void testStopWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml"; final String yaml = "/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -208,4 +209,76 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
}); });
assertThat(workflowRepository.getAll()).isEmpty(); assertThat(workflowRepository.getAll()).isEmpty();
} }
@Test
@DisplayName("Test stop a workflow with one sub workflow task")
public void testStopWorkflow_with_subWorkflowTask_success() {
final String yaml = "/it/stop/workflow_with_sub_workflow_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryWorkflowInstance(context.getWorkflows().get(1)))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
});
assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess());
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.NO);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("sub_logic_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL);
});
final WorkflowDefinition subWorkflowDefinition = context.getWorkflows().get(1);
Assertions
.assertThat(repository.queryWorkflowInstance(subWorkflowDefinition))
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.YES);
});
Assertions
.assertThat(repository.queryTaskInstance(subWorkflowDefinition))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL);
});
});
assertThat(workflowRepository.getAll()).isEmpty();
}
} }

4
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java

@ -68,7 +68,7 @@ public class WorkflowSchedulingIT extends AbstractMasterIntegrationTest {
public void testSchedulingWorkflow_with_oneSuccessTask() { public void testSchedulingWorkflow_with_oneSuccessTask() {
final String yaml = "/it/scheduling/workflow_with_one_fake_task_success.yaml"; final String yaml = "/it/scheduling/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final Schedule schedule = Schedule.builder() final Schedule schedule = Schedule.builder()
.processDefinitionCode(workflow.getCode()) .processDefinitionCode(workflow.getCode())
@ -89,7 +89,7 @@ public class WorkflowSchedulingIT extends AbstractMasterIntegrationTest {
WorkflowOperator.WorkflowSchedulingDTO workflowSchedulingDTO = WorkflowOperator.WorkflowSchedulingDTO.builder() WorkflowOperator.WorkflowSchedulingDTO workflowSchedulingDTO = WorkflowOperator.WorkflowSchedulingDTO.builder()
.project(context.getProject()) .project(context.getProject())
.workflow(context.getWorkflow()) .workflow(context.getWorkflows().get(0))
.schedule(schedule) .schedule(schedule)
.build(); .build();

135
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java

@ -24,6 +24,7 @@ import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition; import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.enums.DataType; import org.apache.dolphinscheduler.plugin.task.api.enums.DataType;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
@ -72,21 +73,21 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneSuccessTask() { public void testStartWorkflow_with_oneSuccessTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_success.yaml"; final String yaml = "/it/start/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.manualTriggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState()) .matches(
.isEqualTo(WorkflowExecutionStatus.SUCCESS)); workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.SUCCESS);
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflow))
.satisfiesExactly(taskInstance -> { .satisfiesExactly(taskInstance -> {
@ -98,12 +99,116 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
assertThat(workflowRepository.getAll()).isEmpty(); assertThat(workflowRepository.getAll()).isEmpty();
} }
@Test
@DisplayName("Test start a workflow with one sub workflow task(A) success")
public void testStartWorkflow_with_subWorkflowTask_success() {
final String yaml = "/it/start/workflow_with_sub_workflow_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition parentWorkflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(parentWorkflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.matches(
workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.SUCCESS)
.matches(
workflowInstance -> workflowInstance.getIsSubProcess() == Flag.NO);
final List<WorkflowInstance> subWorkflowInstance =
repository.queryWorkflowInstance(context.getWorkflows().get(1));
Assertions
.assertThat(subWorkflowInstance)
.hasSize(1)
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.SUCCESS);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.YES);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("sub_logic_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
});
Assertions
.assertThat(repository.queryTaskInstance(subWorkflowInstance.get(0).getId()))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("fake_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
});
});
assertThat(workflowRepository.getAll()).isEmpty();
}
@Test
@DisplayName("Test start a workflow with one sub workflow task(A) failed")
public void testStartWorkflow_with_subWorkflowTask_failed() {
final String yaml = "/it/start/workflow_with_sub_workflow_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition parentWorkflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(parentWorkflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.matches(
workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.FAILURE)
.matches(
workflowInstance -> workflowInstance.getIsSubProcess() == Flag.NO);
final List<WorkflowInstance> subWorkflowInstance =
repository.queryWorkflowInstance(context.getWorkflows().get(1));
Assertions
.assertThat(subWorkflowInstance)
.hasSize(1)
.satisfiesExactly(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.FAILURE);
assertThat(workflowInstance.getIsSubProcess()).isEqualTo(Flag.YES);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("sub_logic_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE);
});
Assertions
.assertThat(repository.queryTaskInstance(subWorkflowInstance.get(0).getId()))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("fake_task");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE);
});
});
assertThat(workflowRepository.getAll()).isEmpty();
}
@Test @Test
@DisplayName("Test start a workflow which using workflow params") @DisplayName("Test start a workflow which using workflow params")
public void testStartWorkflow_usingWorkflowParam() { public void testStartWorkflow_usingWorkflowParam() {
final String yaml = "/it/start/workflow_with_global_param.yaml"; final String yaml = "/it/start/workflow_with_global_param.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -139,7 +244,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_usingCommandParam() { public void testStartWorkflow_usingCommandParam() {
final String yaml = "/it/start/workflow_with_global_param.yaml"; final String yaml = "/it/start/workflow_with_global_param.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder() final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
.commandParams(Lists.newArrayList(Property.builder() .commandParams(Lists.newArrayList(Property.builder()
@ -184,7 +289,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneFailedTask() { public void testStartWorkflow_with_oneFailedTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_failed.yaml"; final String yaml = "/it/start/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -215,7 +320,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneFailedTaskWithRetry() { public void testStartWorkflow_with_oneFailedTaskWithRetry() {
final String yaml = "/it/start/workflow_with_one_fake_task_failed_with_retry.yaml"; final String yaml = "/it/start/workflow_with_one_fake_task_failed_with_retry.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -268,7 +373,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoSerialSuccessTask() { public void testStartWorkflow_with_twoSerialSuccessTask() {
String yaml = "/it/start/workflow_with_two_serial_fake_task_success.yaml"; String yaml = "/it/start/workflow_with_two_serial_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -307,7 +412,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoSerialFailedTask() { public void testStartWorkflow_with_twoSerialFailedTask() {
final String yaml = "/it/start/workflow_with_two_serial_fake_task_failed.yaml"; final String yaml = "/it/start/workflow_with_two_serial_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -338,7 +443,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoParallelSuccessTask() { public void testStartWorkflow_with_twoParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_two_parallel_fake_task_success.yaml"; final String yaml = "/it/start/workflow_with_two_parallel_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -376,7 +481,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoParallelFailedTask() { public void testStartWorkflow_with_twoParallelFailedTask() {
final String yaml = "/it/start/workflow_with_two_parallel_fake_task_failed.yaml"; final String yaml = "/it/start/workflow_with_two_parallel_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -415,7 +520,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_threeParallelSuccessTask() { public void testStartWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.yaml"; final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
.workflowDefinition(workflow) .workflowDefinition(workflow)
@ -482,7 +587,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflowFromStartNodes_with_threeParallelSuccessTask() { public void testStartWorkflowFromStartNodes_with_threeParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.yaml"; final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml); final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflow(); final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder() final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
.startNodes(Lists.newArrayList(6L)) .startNodes(Lists.newArrayList(6L))

22
dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_failed - name: workflow_with_one_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

111
dolphinscheduler-master/src/test/resources/it/pause/workflow_with_sub_workflow_task_success.yaml

@ -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.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 3
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_A
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_B
code: 2
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

22
dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_three_parallel_three_fake_task_success - name: workflow_with_three_parallel_three_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel success tasks description: This is a fake workflow with two parallel success tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A1 - name: A1

22
dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml

@ -73,17 +73,17 @@ taskInstances:
varPool: '[]' varPool: '[]'
taskExecuteType: BATCH taskExecuteType: BATCH
workflow: workflows:
name: workflow_with_two_serial_fake_task_success - name: workflow_with_two_serial_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two serial tasks description: This is a fake workflow with two serial tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

111
dolphinscheduler-master/src/test/resources/it/recover_paused/workflow_with_sub_workflow_task_success.yaml

@ -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.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 3
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_A
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_B
code: 2
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

111
dolphinscheduler-master/src/test/resources/it/recover_stopped/workflow_with_sub_workflow_task_success.yaml

@ -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.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 3
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_A
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_B
code: 2
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

22
dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
workflowInstance: workflowInstance:
id: 1 id: 1

22
dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml

@ -74,17 +74,17 @@ taskInstances:
varPool: '[]' varPool: '[]'
taskExecuteType: BATCH taskExecuteType: BATCH
workflow: workflows:
name: workflow_with_two_parallel_fake_task_success - name: workflow_with_two_parallel_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel success tasks description: This is a fake workflow with two parallel success tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

24
dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml

@ -24,18 +24,18 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
globalParams: '[{"prop":"name","value":"workflowParam","direct":"IN","type":"VARCHAR"}]' globalParams: '[{"prop":"name","value":"workflowParam","direct":"IN","type":"VARCHAR"}]'
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_failed - name: workflow_with_one_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_failed - name: workflow_with_one_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

91
dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_failed.yaml

@ -0,0 +1,91 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 2
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"xx"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

91
dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_success.yaml

@ -0,0 +1,91 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 2
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"echo success"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_three_parallel_three_fake_task_success - name: workflow_with_three_parallel_three_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel success tasks description: This is a fake workflow with two parallel success tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A1 - name: A1

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_two_parallel_fake_task_failed - name: workflow_with_two_parallel_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel failed tasks description: This is a fake workflow with two parallel failed tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_two_parallel_fake_task_success - name: workflow_with_two_parallel_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel success tasks description: This is a fake workflow with two parallel success tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_two_serial_fake_task_failed - name: workflow_with_two_serial_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two serial tasks description: This is a fake workflow with two serial tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml

@ -23,17 +23,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_two_serial_fake_task_success - name: workflow_with_two_serial_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two serial tasks description: This is a fake workflow with two serial tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_failed - name: workflow_with_one_fake_task_failed
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

22
dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_one_fake_task_success - name: workflow_with_one_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with single task description: This is a fake workflow with single task
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A - name: A

111
dolphinscheduler-master/src/test/resources/it/stop/workflow_with_sub_workflow_task_success.yaml

@ -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.
#
project:
name: MasterIntegrationTest
code: 1
description: This is a fake project
userId: 1
userName: admin
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
workflows:
- name: parent_workflow
code: 2
version: 1
projectCode: 1
description: This is a workflow with sub workflow task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
- name: child_workflow
code: 1
version: 1
projectCode: 1
description: This is a workflow with a fake task
releaseState: ONLINE
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
userId: 1
executionType: PARALLEL
tasks:
- name: sub_logic_task
code: 3
version: 1
projectCode: 1
userId: 1
taskType: SUB_PROCESS
taskParams: '{"localParams":[],"resourceList":[],"processDefinitionCode":1}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_A
code: 1
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
- name: fake_task_B
code: 2
version: 1
projectCode: 1
userId: 1
taskType: LogicFakeTask
taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 5"}'
workerGroup: default
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
taskExecuteType: BATCH
taskRelations:
- projectCode: 1
processDefinitionCode: 2
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2
postTaskVersion: 1
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00

22
dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml

@ -24,17 +24,17 @@ project:
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
workflow: workflows:
name: workflow_with_three_parallel_three_fake_task_success - name: workflow_with_three_parallel_three_fake_task_success
code: 1 code: 1
version: 1 version: 1
projectCode: 1 projectCode: 1
description: This is a fake workflow with two parallel success tasks description: This is a fake workflow with two parallel success tasks
releaseState: ONLINE releaseState: ONLINE
createTime: 2024-08-12 00:00:00 createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00 updateTime: 2021-08-12 00:00:00
userId: 1 userId: 1
executionType: PARALLEL executionType: PARALLEL
tasks: tasks:
- name: A1 - name: A1

Loading…
Cancel
Save