diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
index 437d10a386..f87667b2e9 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
+++ b/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.Priority;
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.utils.TaskTypeUtils;
import java.io.Serializable;
import java.util.Date;
-import java.util.concurrent.TimeUnit;
import lombok.Data;
@@ -273,50 +270,4 @@ public class TaskInstance implements Serializable {
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;
- }
-
}
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java
index efa4e3d4a0..eb44833f7b 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkflowInstanceRelation.java
@@ -17,15 +17,19 @@
package org.apache.dolphinscheduler.dao.entity;
-import java.util.Objects;
-
+import lombok.AllArgsConstructor;
+import lombok.Builder;
import lombok.Data;
+import lombok.NoArgsConstructor;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
@TableName("t_ds_relation_process_instance")
public class WorkflowInstanceRelation {
@@ -50,35 +54,4 @@ public class WorkflowInstanceRelation {
*/
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;
- }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RecoverFailureTaskCommandHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RecoverFailureTaskCommandHandler.java
index 106ac1c55c..6b712de492 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RecoverFailureTaskCommandHandler.java
+++ b/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.graph.IWorkflowGraph;
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.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.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
import java.util.ArrayList;
-import java.util.Date;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -71,6 +72,9 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
@Autowired
private ApplicationContext applicationContext;
+ @Autowired
+ private TaskInstanceFactories taskInstanceFactories;
+
/**
* Generate the recover workflow instance.
*
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.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());
workflowInstance.setCommandType(command.getCommandType());
- workflowInstance.setStartTime(new Date());
- workflowInstance.setRestartTime(new Date());
- workflowInstance.setEndTime(null);
- workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1);
workflowInstanceDao.updateById(workflowInstance);
workflowExecuteContextBuilder.setWorkflowInstance(workflowInstance);
@@ -133,13 +133,14 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors);
};
- final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder()
- .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
- .onWorkflowGraph(workflowGraph)
- .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
- .doVisitFunction(taskExecutionRunnableCreator)
- .build();
- workflowGraphBfsVisitor.visit();
+ final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
+ WorkflowGraphTopologyLogicalVisitor.builder()
+ .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
+ .onWorkflowGraph(workflowGraph)
+ .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
+ .doVisitFunction(taskExecutionRunnableCreator)
+ .build();
+ workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
}
@@ -155,42 +156,52 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
.stream()
.collect(Collectors.toMap(TaskInstance::getName, Function.identity()));
- final Set needRecreateTasks = taskInstanceMap.values()
- .stream()
- .filter(this::isTaskNeedRecreate)
- .map(TaskInstance::getName)
- .collect(Collectors.toSet());
-
final IWorkflowGraph workflowGraph = workflowExecuteContextBuilder.getWorkflowGraph();
+
+ final Set needRecoverTasks = new HashSet<>();
+ final Set markInvalidTasks = new HashSet<>();
final BiConsumer> historyTaskInstanceMarker = (task, successors) -> {
- boolean isTaskNeedRecreate = needRecreateTasks.contains(task) || workflowGraph.getPredecessors(task)
- .stream()
- .anyMatch(needRecreateTasks::contains);
- // 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 the parent is need recover
+ // Then the task should mark as invalid, and it's child should be mark as invalidated.
+ if (markInvalidTasks.contains(task)) {
if (taskInstanceMap.containsKey(task)) {
taskInstanceDao.markTaskInstanceInvalid(Lists.newArrayList(taskInstanceMap.get(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.
- if (isTaskNeedRecover(taskInstanceMap.get(task))) {
- final TaskInstance taskInstance = taskInstanceMap.get(task);
- taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
- taskInstanceDao.upsertTaskInstance(taskInstance);
+
+ final TaskInstance taskInstance = taskInstanceMap.get(task);
+ if (taskInstance == null) {
+ return;
+ }
+
+ if (isTaskNeedRecreate(taskInstance) || isTaskCanRecover(taskInstance)) {
+ needRecoverTasks.add(task);
+ markInvalidTasks.addAll(successors);
}
};
- final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder()
- .onWorkflowGraph(workflowGraph)
- .taskDependType(workflowInstance.getTaskDependType())
- .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
- .doVisitFunction(historyTaskInstanceMarker)
- .build();
- workflowGraphBfsVisitor.visit();
+ final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
+ WorkflowGraphTopologyLogicalVisitor.builder()
+ .onWorkflowGraph(workflowGraph)
+ .taskDependType(workflowInstance.getTaskDependType())
+ .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
+ .doVisitFunction(historyTaskInstanceMarker)
+ .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());
}
@@ -199,17 +210,34 @@ public class RecoverFailureTaskCommandHandler extends AbstractCommandHandler {
* If the task state is FAILURE and KILL, then will mark the task invalid and recreate the task.
*/
private boolean isTaskNeedRecreate(final TaskInstance taskInstance) {
+ if (taskInstance == null) {
+ return false;
+ }
return taskInstance.getState() == TaskExecutionStatus.FAILURE
|| 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) {
return false;
}
return taskInstance.getState() == TaskExecutionStatus.PAUSE;
}
+ private TaskInstance createRecoverTaskInstance(final TaskInstance taskInstance) {
+ return taskInstanceFactories.pauseRecoverTaskInstanceFactory()
+ .builder()
+ .withTaskInstance(taskInstance)
+ .build();
+ }
+
@Override
public CommandType commandType() {
return CommandType.START_FAILURE_TASK_PROCESS;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java
index eb81c9fd58..27ebeb9796 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java
+++ b/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.engine.graph.IWorkflowGraph;
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.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
@@ -107,13 +107,14 @@ public class RunWorkflowCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors);
};
- final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder()
- .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
- .onWorkflowGraph(workflowGraph)
- .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
- .doVisitFunction(taskExecutionRunnableCreator)
- .build();
- workflowGraphBfsVisitor.visit();
+ final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
+ WorkflowGraphTopologyLogicalVisitor.builder()
+ .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
+ .onWorkflowGraph(workflowGraph)
+ .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
+ .doVisitFunction(taskExecutionRunnableCreator)
+ .build();
+ workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/WorkflowFailoverCommandHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/WorkflowFailoverCommandHandler.java
index 3b4fbc1d8c..f47f43adc1 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/WorkflowFailoverCommandHandler.java
+++ b/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.graph.IWorkflowGraph;
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.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory;
@@ -129,13 +129,14 @@ public class WorkflowFailoverCommandHandler extends AbstractCommandHandler {
workflowExecutionGraph.addEdge(task, successors);
};
- final WorkflowGraphBfsVisitor workflowGraphBfsVisitor = WorkflowGraphBfsVisitor.builder()
- .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
- .onWorkflowGraph(workflowGraph)
- .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
- .doVisitFunction(taskExecutionRunnableCreator)
- .build();
- workflowGraphBfsVisitor.visit();
+ final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
+ WorkflowGraphTopologyLogicalVisitor.builder()
+ .taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
+ .onWorkflowGraph(workflowGraph)
+ .fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
+ .doVisitFunction(taskExecutionRunnableCreator)
+ .build();
+ workflowGraphTopologyLogicalVisitor.visit();
workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphBfsVisitor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphTopologyLogicalVisitor.java
similarity index 63%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphBfsVisitor.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphTopologyLogicalVisitor.java
index fa15d0cff4..24efa3baf3 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowGraphBfsVisitor.java
+++ b/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 org.apache.dolphinscheduler.common.enums.TaskDependType;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.commons.collections4.CollectionUtils;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.function.BiConsumer;
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 startNodes;
+ private final TaskDependType taskDependType;
- private BiConsumer> visitFunction;
+ private final Set startNodes;
- private WorkflowGraphBfsVisitor(WorkflowGraphBfsVisitorBuilder workflowGraphBfsVisitorBuilder) {
+ private final BiConsumer> visitFunction;
+
+ private WorkflowGraphTopologyLogicalVisitor(WorkflowGraphBfsVisitorBuilder workflowGraphBfsVisitorBuilder) {
this.taskDependType = workflowGraphBfsVisitorBuilder.taskDependType;
this.workflowGraph = checkNotNull(workflowGraphBfsVisitorBuilder.workflowGraph);
this.visitFunction = checkNotNull(workflowGraphBfsVisitorBuilder.visitFunction);
@@ -72,60 +76,76 @@ public class WorkflowGraphBfsVisitor {
}
/**
- * visit start nodes only
+ * Visit start nodes only.
*/
private void visitStartNodesOnly() {
- startNodes.forEach(startNode -> {
- final Set realSuccessors = workflowGraph.getSuccessors(startNode)
- .stream()
- .filter(startNode::contains)
- .collect(Collectors.toSet());
- visitFunction.accept(startNode, realSuccessors);
- });
+ doVisitationInSubGraph(Sets.newHashSet(startNodes));
}
/**
- * 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() {
final LinkedList bootstrapTaskCodes = new LinkedList<>(startNodes);
- final Set visited = new HashSet<>();
+ final Set subGraphNodes = new HashSet<>();
while (!bootstrapTaskCodes.isEmpty()) {
String taskName = bootstrapTaskCodes.removeFirst();
- if (visited.contains(taskName)) {
+ if (subGraphNodes.contains(taskName)) {
continue;
}
- visited.add(taskName);
+ subGraphNodes.add(taskName);
final Set successors = workflowGraph.getPredecessors(taskName);
bootstrapTaskCodes.addAll(successors);
}
- visited.forEach(taskName -> {
- Set realSuccessors = workflowGraph.getSuccessors(taskName)
- .stream()
- .filter(visited::contains)
- .collect(Collectors.toSet());
- visitFunction.accept(taskName, realSuccessors);
- });
+ doVisitationInSubGraph(subGraphNodes);
}
/**
- * 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() {
final LinkedList bootstrapTaskCodes = new LinkedList<>(startNodes);
- final Set visited = new HashSet<>();
-
+ final Set subGraphNodes = new HashSet<>();
while (!bootstrapTaskCodes.isEmpty()) {
String taskName = bootstrapTaskCodes.removeFirst();
- if (visited.contains(taskName)) {
+ if (subGraphNodes.contains(taskName)) {
continue;
}
- visited.add(taskName);
+ subGraphNodes.add(taskName);
final Set successors = workflowGraph.getSuccessors(taskName);
- visitFunction.accept(taskName, successors);
bootstrapTaskCodes.addAll(successors);
}
+ doVisitationInSubGraph(subGraphNodes);
+ }
+
+ private void doVisitationInSubGraph(Set subGraphNodes) {
+ // visit from the workflow graph by topology
+ // If the node is not in the subGraph, then skip it.
+ Map inDegreeMap = workflowGraph.getAllTaskNodes()
+ .stream()
+ .collect(Collectors.toMap(TaskDefinition::getName,
+ taskDefinition -> workflowGraph.getPredecessors(taskDefinition.getName()).size()));
+ final LinkedList 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 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 {
@@ -158,8 +178,8 @@ public class WorkflowGraphBfsVisitor {
return this;
}
- public WorkflowGraphBfsVisitor build() {
- return new WorkflowGraphBfsVisitor(this);
+ public WorkflowGraphTopologyLogicalVisitor build() {
+ return new WorkflowGraphTopologyLogicalVisitor(this);
}
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/FailedRecoverTaskInstanceFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/FailedRecoverTaskInstanceFactory.java
new file mode 100644
index 0000000000..479208faba
--- /dev/null
+++ b/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 {
+
+ @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);
+ }
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/PauseRecoverTaskInstanceFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/PauseRecoverTaskInstanceFactory.java
new file mode 100644
index 0000000000..77d9acdd5d
--- /dev/null
+++ b/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 {
+
+ @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);
+ }
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java
index 0f99a2447e..36dc13b085 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskInstanceFactories.java
@@ -29,6 +29,12 @@ public class TaskInstanceFactories {
@Autowired
private RetryTaskInstanceFactory retryTaskInstanceFactory;
+ @Autowired
+ private FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory;
+
+ @Autowired
+ private PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory;
+
@Autowired
private FailoverTaskInstanceFactory failoverTaskInstanceFactory;
@@ -40,8 +46,15 @@ public class TaskInstanceFactories {
return retryTaskInstanceFactory;
}
+ public FailedRecoverTaskInstanceFactory failedRecoverTaskInstanceFactory() {
+ return failedRecoverTaskInstanceFactory;
+ }
+
public FailoverTaskInstanceFactory failoverTaskInstanceFactory() {
return failoverTaskInstanceFactory;
}
+ public PauseRecoverTaskInstanceFactory pauseRecoverTaskInstanceFactory() {
+ return pauseRecoverTaskInstanceFactory;
+ }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverFailureTaskTrigger.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverFailureTaskTrigger.java
index cc9ba05b37..bb1f5a3101 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverFailureTaskTrigger.java
+++ b/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;
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.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverFailureTasksRequest;
@@ -34,7 +35,14 @@ public class WorkflowInstanceRecoverFailureTaskTrigger
@Override
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
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverSuspendTaskTrigger.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverSuspendTaskTrigger.java
index 56ee9b4a5c..5ac3d8014e 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowInstanceRecoverSuspendTaskTrigger.java
+++ b/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;
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.WorkflowInstance;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceRecoverSuspendTasksRequest;
@@ -34,7 +35,14 @@ public class WorkflowInstanceRecoverSuspendTaskTrigger
@Override
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
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java
index 121c5ae6e1..59e18993da 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java
+++ b/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.plugin.task.api.utils.LogUtils;
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.MasterTaskExecutorHolder;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorThreadPoolManager;
@@ -54,16 +53,10 @@ public class LogicITaskInstanceKillOperationFunction
}
try {
masterTaskExecutor.cancelTask();
- // todo: if we remove success then we don't need to cancel?
- masterTaskExecutorThreadPool.removeMasterTaskExecutor(masterTaskExecutor);
return LogicTaskKillResponse.success();
} catch (MasterTaskExecuteException e) {
log.error("Cancel MasterTaskExecuteRunnable failed ", e);
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 {
LogUtils.removeTaskInstanceIdMDC();
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java
index e7babaa417..2c5bf4e548 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java
+++ b/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.
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext);
break;
+ case PAUSE:
+ asyncTaskCallbackFunction.executePause();
+ break;
+ case KILL:
+ asyncTaskCallbackFunction.executeKilled();
+ break;
case SUCCESS:
asyncTaskCallbackFunction.executeSuccess();
break;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java
index 2cc23c1248..df9ed8f5b5 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java
+++ b/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 executePause();
+
+ void executeKilled();
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java
index 6c83a3ccb9..ae7a480cd0 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java
+++ b/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);
}
+ @Override
+ public void executePause() {
+ asyncMasterTaskExecuteRunnable.getTaskExecutionContext()
+ .setCurrentExecutionStatus(TaskExecutionStatus.PAUSE);
+ executeFinished();
+ }
+
+ @Override
+ public void executeKilled() {
+ asyncMasterTaskExecuteRunnable.getTaskExecutionContext()
+ .setCurrentExecutionStatus(TaskExecutionStatus.KILL);
+ executeFinished();
+ }
+
private void executeFinished() {
TaskInstanceLogHeader.printFinalizeTaskHeader();
int taskInstanceId = asyncMasterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId();
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java
index 3b2f62a2d1..2a34c4adc8 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java
@@ -33,6 +33,10 @@ public interface AsyncTaskExecuteFunction {
RUNNING,
+ PAUSE,
+
+ KILL,
+
SUCCESS,
FAILED,
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java
index 84c342c074..8fa6002463 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java
+++ b/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;
+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.dynamic.DynamicLogicTask;
-import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask;
import java.util.Set;
@@ -42,7 +42,7 @@ public class MasterTaskExecutorFactoryBuilder {
private static final Set ASYNC_TASK_TYPE = Sets.newHashSet(
DependentLogicTask.TASK_TYPE,
- SubWorkflowLogicTask.TASK_TYPE,
+ SubWorkflowLogicTaskChannelFactory.NAME,
DynamicLogicTask.TASK_TYPE);
public MasterTaskExecutorFactory extends MasterTaskExecutor> createMasterTaskExecutorFactory(String taskType) {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java
index 0d190659f7..2b9d5728a1 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java
+++ b/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.plugin.task.api.TaskExecutionContext;
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;
@@ -37,15 +35,6 @@ public abstract class BaseAsyncLogicTask implement
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
public TaskExecutionContext getTaskExecutionContext() {
return taskExecutionContext;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
index b567674052..70ac2ba10b 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
@@ -18,22 +18,18 @@
package org.apache.dolphinscheduler.server.master.runner.task.dependent;
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.TaskDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
-import java.util.Date;
-
import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference;
@@ -51,6 +47,8 @@ public class DependentLogicTask extends BaseAsyncLogicTask
private final IWorkflowExecutionRunnable workflowExecutionRunnable;
+ private DependentAsyncTaskExecuteFunction dependentAsyncTaskExecuteFunction;
+
public DependentLogicTask(TaskExecutionContext taskExecutionContext,
ProjectDao projectDao,
WorkflowDefinitionDao workflowDefinitionDao,
@@ -72,34 +70,24 @@ public class DependentLogicTask extends BaseAsyncLogicTask
@Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
- return new DependentAsyncTaskExecuteFunction(taskExecutionContext,
+ dependentAsyncTaskExecuteFunction = new DependentAsyncTaskExecuteFunction(taskExecutionContext,
taskParameters,
projectDao,
workflowDefinitionDao,
taskDefinitionDao,
taskInstanceDao,
workflowInstanceDao);
+ return dependentAsyncTaskExecuteFunction;
}
@Override
public void pause() throws MasterTaskExecuteException {
- if (workflowExecutionRunnable == null) {
- log.error("Cannot find the WorkflowExecuteRunnable");
- return;
- }
- TaskInstance taskInstance = workflowExecutionRunnable
- .getWorkflowExecuteContext()
- .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();
+ // todo: support pause
+ }
+
+ @Override
+ public void kill() throws MasterTaskExecuteException {
+ // todo: support kill
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
index 492cb4cfb7..547360b539 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
@@ -250,6 +250,11 @@ public class DynamicLogicTask extends BaseAsyncLogicTask {
return dynamicInputParameters;
}
+ @Override
+ public void pause() throws MasterTaskExecuteException {
+ // todo: support pause
+ }
+
@Override
public void kill() {
try {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java
index d85edf8e3c..8938469762 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java
+++ b/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.repository.WorkflowInstanceDao;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
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 final TaskExecutionContext taskExecutionContext;
private final WorkflowInstanceDao workflowInstanceDao;
- private WorkflowInstance subWorkflowInstance;
- private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
+ private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
- public SubWorkflowAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext,
- WorkflowInstanceDao workflowInstanceDao) {
- this.taskExecutionContext = taskExecutionContext;
+ public SubWorkflowAsyncTaskExecuteFunction(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext,
+ final WorkflowInstanceDao workflowInstanceDao) {
+ this.subWorkflowLogicTaskRuntimeContext = subWorkflowLogicTaskRuntimeContext;
this.workflowInstanceDao = workflowInstanceDao;
}
@Override
public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
+ final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
+ final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId);
if (subWorkflowInstance == null) {
- createSubWorkflowInstanceIfAbsent();
+ log.info("Cannot find the SubWorkflow instance: {}, maybe it has been deleted", subWorkflowInstanceId);
+ return AsyncTaskExecutionStatus.FAILED;
}
- if (subWorkflowInstance == null) {
- log.info("The sub workflow instance doesn't created");
- return AsyncTaskExecutionStatus.RUNNING;
- }
- subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstance.getId());
- if (subWorkflowInstance != null && subWorkflowInstance.getState().isFinished()) {
- return subWorkflowInstance.getState().isSuccess() ? AsyncTaskExecutionStatus.SUCCESS
- : AsyncTaskExecutionStatus.FAILED;
+ switch (subWorkflowInstance.getState()) {
+ case PAUSE:
+ return AsyncTaskExecutionStatus.PAUSE;
+ case STOP:
+ return AsyncTaskExecutionStatus.KILL;
+ case SUCCESS:
+ return AsyncTaskExecutionStatus.SUCCESS;
+ 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
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java
new file mode 100644
index 0000000000..69109f09dc
--- /dev/null
+++ b/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);
+ }
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
index e8b791155b..868674d8c6 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
@@ -17,36 +17,30 @@
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.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
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.WorkflowInstanceDao;
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
+import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceMapDao;
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.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.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.parameters.SubProcessParameters;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
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.message.LogicTaskInstanceExecutionEventSenderManager;
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 org.springframework.context.ApplicationContext;
@@ -56,19 +50,11 @@ import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j
public class SubWorkflowLogicTask extends BaseAsyncLogicTask {
- public static final String TASK_TYPE = "SUB_PROCESS";
-
- private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager;
-
- private final WorkflowInstanceDao workflowInstanceDao;
-
- private final WorkflowDefinitionDao workflowDefinitionDao;
-
- private final CommandDao commandDao;
+ private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
private final IWorkflowExecutionRunnable workflowExecutionRunnable;
- private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
+ private final ApplicationContext applicationContext;
public SubWorkflowLogicTask(final TaskExecutionContext taskExecutionContext,
final IWorkflowExecutionRunnable workflowExecutionRunnable,
@@ -76,141 +62,165 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask() {
}));
- 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.subWorkflowLogicTaskRuntimeContext = initializeSubWorkflowLogicTaskRuntimeContext();
- taskExecutionContext.setAppIds(JSONUtils.toJsonString(subWorkflowLogicTaskRuntimeContext));
- logicTaskInstanceExecutionEventSenderManager.runningEventSender().sendMessage(taskExecutionContext);
+ this.applicationContext = applicationContext;
+ this.subWorkflowLogicTaskRuntimeContext = JSONUtils.parseObject(
+ taskExecutionContext.getAppIds(),
+ SubWorkflowLogicTaskRuntimeContext.class);
}
@Override
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
public void pause() throws MasterTaskExecuteException {
- WorkflowInstance subWorkflowInstance =
- workflowInstanceDao.querySubWorkflowInstanceByParentId(taskExecutionContext.getProcessInstanceId(),
- taskExecutionContext.getTaskInstanceId());
-
- try {
- WorkflowInstancePauseResponse pauseResponse = Clients
- .withService(IWorkflowControlClient.class)
- .withHost(subWorkflowInstance.getHost())
- .pauseWorkflowInstance(new WorkflowInstancePauseRequest(subWorkflowInstance.getId()));
- if (pauseResponse.isSuccess()) {
- log.info("Pause sub workflowInstance: {}", subWorkflowInstance.getName() + " success");
- } else {
- throw new MasterTaskExecuteException(
- "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);
+ if (subWorkflowLogicTaskRuntimeContext == null) {
+ log.info("subWorkflowLogicTaskRuntimeContext is null cannot pause");
+ return;
+ }
+ final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
+ final WorkflowInstancePauseResponse pauseResponse = applicationContext
+ .getBean(SubWorkflowControlClient.class)
+ .pauseWorkflowInstance(new WorkflowInstancePauseRequest(subWorkflowInstanceId));
+ if (pauseResponse.isSuccess()) {
+ log.info("Pause sub workflowInstance: id={}", subWorkflowInstanceId + " success");
+ } else {
+ log.info("Pause sub workflowInstance: id={} failed with response: {}", subWorkflowInstanceId,
+ pauseResponse);
}
}
@Override
- public void kill() {
- WorkflowInstance subWorkflowInstance =
- workflowInstanceDao.querySubWorkflowInstanceByParentId(taskExecutionContext.getProcessInstanceId(),
- taskExecutionContext.getTaskInstanceId());
- if (subWorkflowInstance == null) {
- log.info("SubWorkflow instance is null");
+ public void kill() throws MasterTaskExecuteException {
+ if (subWorkflowLogicTaskRuntimeContext == null) {
+ log.info("subWorkflowLogicTaskRuntimeContext is null cannot kill");
return;
}
- try {
- WorkflowInstanceStopResponse stopResponse = Clients
- .withService(IWorkflowControlClient.class)
- .withHost(subWorkflowInstance.getHost())
- .stopWorkflowInstance(new WorkflowInstanceStopRequest(subWorkflowInstance.getId()));
- if (stopResponse.isSuccess()) {
- log.info("Kill sub workflowInstance: {}", subWorkflowInstance.getName() + " success");
- } else {
- 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);
+ final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
+ final WorkflowInstanceStopResponse stopResponse = applicationContext
+ .getBean(SubWorkflowControlClient.class)
+ .stopWorkflowInstance(new WorkflowInstanceStopRequest(subWorkflowInstanceId));
+ if (stopResponse.isSuccess()) {
+ log.info("Kill sub workflowInstance: id={}", subWorkflowInstanceId + " success");
+ } else {
+ log.info("Kill sub workflowInstance: id={} failed with response: {}", subWorkflowInstanceId, stopResponse);
}
}
- private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowLogicTaskRuntimeContext() {
- if (taskExecutionContext.isFailover() && StringUtils.isNotEmpty(taskExecutionContext.getAppIds())) {
- return JSONUtils.parseObject(taskExecutionContext.getAppIds(), SubWorkflowLogicTaskRuntimeContext.class);
+ private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowInstance() {
+ // todo: doFailover if the runtime context is not null and task is generated by failover
+
+ 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...
- 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:
+
+ switch (workflowExecutionRunnable.getWorkflowInstance().getCommandType()) {
case RECOVER_SUSPENDED_PROCESS:
- return createSubWorkflowInstanceWithWorkflowInstance();
+ return recoverFromSuspendTasks();
+ case START_FAILURE_TASK_PROCESS:
+ return recoverFromFailedTasks();
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 WorkflowDefinition subWorkflowDefinition = applicationContext.getBean(WorkflowDefinitionDao.class)
+ .queryByCode(taskParameters.getProcessDefinitionCode())
+ .orElseThrow(() -> new IllegalArgumentException(
+ "Cannot find the sub workflow definition: " + taskParameters.getProcessDefinitionCode()));
+
final ICommandParam commandParam =
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 Command command = Command.builder()
- .commandType(workflowInstance.getCommandType())
- .processDefinitionCode(subWorkflowDefinition.getCode())
- .processDefinitionVersion(subWorkflowDefinition.getVersion())
- .executorId(workflowInstance.getExecutorId())
- .commandParam(JSONUtils.toJsonString(runWorkflowCommandParam))
- .taskDependType(TaskDependType.TASK_POST)
+ final WorkflowManualTriggerRequest workflowManualTriggerRequest = WorkflowManualTriggerRequest.builder()
+ .userId(taskExecutionContext.getExecutorId())
+ .workflowDefinitionCode(subWorkflowDefinition.getCode())
+ .workflowDefinitionVersion(subWorkflowDefinition.getVersion())
.failureStrategy(workflowInstance.getFailureStrategy())
.warningType(workflowInstance.getWarningType())
.warningGroupId(workflowInstance.getWarningGroupId())
- .startTime(new Date())
- .processInstancePriority(workflowInstance.getProcessInstancePriority())
- .updateTime(new Date())
- .workerGroup(taskExecutionContext.getWorkerGroup())
- .tenantCode(taskExecutionContext.getTenantCode())
- .dryRun(taskExecutionContext.getDryRun())
- .testFlag(taskExecutionContext.getTestFlag())
- .build();
- commandDao.insert(command);
- return SubWorkflowLogicTaskRuntimeContext.builder()
- .subWorkflowCommandId(command.getId())
+ .workflowInstancePriority(workflowInstance.getProcessInstancePriority())
+ .workerGroup(workflowInstance.getWorkerGroup())
+ .tenantCode(workflowInstance.getTenantCode())
+ .environmentCode(workflowInstance.getEnvironmentCode())
+ // todo: transport varpool and local params
+ .startParamList(commandParam.getCommandParams())
+ .dryRun(Flag.of(workflowInstance.getDryRun()))
+ .testFlag(Flag.of(workflowInstance.getTestFlag()))
.build();
+ final Integer subWorkflowInstanceId = applicationContext
+ .getBean(SubWorkflowControlClient.class)
+ .triggerSubWorkflow(workflowManualTriggerRequest);
+ return SubWorkflowLogicTaskRuntimeContext.of(subWorkflowInstanceId);
}
- private SubWorkflowLogicTaskRuntimeContext createSubWorkflowInstanceWithWorkflowInstance() {
- return null;
- }
-
- private WorkflowDefinition getSubWorkflowDefinition() {
- return workflowDefinitionDao.queryByCode(taskParameters.getProcessDefinitionCode()).orElseThrow(
- () -> new IllegalArgumentException(
- "Cannot find the sub workflow definition: " + taskParameters.getProcessDefinitionCode()));
+ private void upsertSubWorkflowRelation() {
+ final WorkflowInstanceMapDao workflowInstanceMapDao = applicationContext.getBean(WorkflowInstanceMapDao.class);
+ WorkflowInstanceRelation workflowInstanceRelation = workflowInstanceMapDao.queryWorkflowMapByParent(
+ taskExecutionContext.getProcessInstanceId(),
+ taskExecutionContext.getTaskInstanceId());
+ if (workflowInstanceRelation == null) {
+ workflowInstanceRelation = WorkflowInstanceRelation.builder()
+ .parentProcessInstanceId(taskExecutionContext.getProcessInstanceId())
+ .parentTaskInstanceId(taskExecutionContext.getTaskInstanceId())
+ .processInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId())
+ .build();
+ workflowInstanceMapDao.insert(workflowInstanceRelation);
+ } else {
+ workflowInstanceRelation
+ .setProcessInstanceId(subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId());
+ workflowInstanceMapDao.updateById(workflowInstanceRelation);
+ }
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java
index 1b750dbc7e..c574885905 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java
@@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
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.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
@@ -52,6 +53,6 @@ public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactor
@Override
public String getTaskType() {
- return SubWorkflowLogicTask.TASK_TYPE;
+ return SubWorkflowLogicTaskChannelFactory.NAME;
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java
index 9d1b327a9c..48d0815735 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java
+++ b/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;
-import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
-
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
@@ -33,9 +31,12 @@ import lombok.NoArgsConstructor;
@NoArgsConstructor
public class SubWorkflowLogicTaskRuntimeContext {
- private Integer subWorkflowCommandId;
-
private Integer subWorkflowInstanceId;
- private WorkflowExecutionStatus subWorkflowExecutionStatus;
+ public static SubWorkflowLogicTaskRuntimeContext of(final Integer subWorkflowInstanceId) {
+ return SubWorkflowLogicTaskRuntimeContext.builder()
+ .subWorkflowInstanceId(subWorkflowInstanceId)
+ .build();
+ }
+
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java
new file mode 100644
index 0000000000..73f32ae43c
--- /dev/null
+++ b/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;
+ }
+
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java
index 15a9182076..6bf1764fca 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java
@@ -37,7 +37,7 @@ public class WorkflowITContext {
private Project project;
- private WorkflowDefinition workflow;
+ private List workflows;
private WorkflowInstance workflowInstance;
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java
index 7c2d9eaef6..6cf23b9549 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java
+++ b/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) {
final WorkflowITContext workflowITContext = YamlFactory.load(yamlPath);
initializeProjectToDB(workflowITContext.getProject());
- initializeWorkflowDefinitionToDB(workflowITContext.getWorkflow());
+ initializeWorkflowDefinitionToDB(workflowITContext.getWorkflows());
initializeTaskDefinitionsToDB(workflowITContext.getTasks());
initializeTaskRelationsToDB(workflowITContext.getTaskRelations());
if (workflowITContext.getWorkflowInstance() != null) {
@@ -99,12 +99,14 @@ public class WorkflowITContextFactory {
workflowInstanceDao.insert(workflowInstance);
}
- private void initializeWorkflowDefinitionToDB(final WorkflowDefinition workflowDefinition) {
- workflowDefinitionDao.insert(workflowDefinition);
- final WorkflowDefinitionLog workflowDefinitionLog = new WorkflowDefinitionLog(workflowDefinition);
- workflowDefinitionLog.setOperator(workflowDefinition.getUserId());
- workflowDefinitionLog.setOperateTime(new Date());
- workflowDefinitionLogDao.insert(workflowDefinitionLog);
+ private void initializeWorkflowDefinitionToDB(final List workflowDefinitions) {
+ for (final WorkflowDefinition workflowDefinition : workflowDefinitions) {
+ workflowDefinitionDao.insert(workflowDefinition);
+ final WorkflowDefinitionLog workflowDefinitionLog = new WorkflowDefinitionLog(workflowDefinition);
+ workflowDefinitionLog.setOperator(workflowDefinition.getUserId());
+ workflowDefinitionLog.setOperateTime(new Date());
+ workflowDefinitionLogDao.insert(workflowDefinitionLog);
+ }
}
private void initializeTaskDefinitionsToDB(final List taskDefinitions) {
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java
index b3ba955e00..cb75951550 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java
+++ b/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.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.WorkflowInstanceRecoverSuspendTasksResponse;
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.WorkflowInstanceStopResponse;
@@ -114,6 +116,15 @@ public class WorkflowOperator {
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) {
final WorkflowInstanceStopRequest workflowInstanceStopRequest =
new WorkflowInstanceStopRequest(workflowInstanceId);
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java
index 6f1fe6ee09..902568aec1 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java
+++ b/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() {
final String yaml = "/it/backfill/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
- final WorkflowDefinition workflow = context.getWorkflow();
+ final WorkflowDefinition workflow = context.getWorkflows().get(0);
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder()
.backfillTimeList(
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java
index d6ed048761..cbb236996d 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java
+++ b/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() {
final String yaml = "/it/failover/running_workflowInstance_with_one_submitted_fake_task.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()));
@@ -92,7 +92,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withDispatchTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_dispatched_fake_task.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()));
@@ -132,7 +132,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withRunningTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_running_fake_task.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()));
@@ -172,7 +172,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_success_fake_task.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()));
@@ -206,7 +206,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_runningWorkflow_withFailedTasks() {
final String yaml = "/it/failover/running_workflowInstance_with_one_failed_fake_task.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()));
@@ -241,7 +241,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withSubmittedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.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()));
@@ -276,7 +276,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withDispatchedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.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()));
@@ -316,7 +316,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_success_fake_task.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()));
@@ -351,7 +351,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withFailedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.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()));
@@ -386,7 +386,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyPauseWorkflow_withPausedTasks() {
final String yaml = "/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.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()));
@@ -421,7 +421,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withSubmittedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.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()));
@@ -456,7 +456,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withDispatchedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.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()));
@@ -496,7 +496,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withSuccessTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_success_fake_task.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()));
@@ -531,7 +531,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withFailedTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.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()));
@@ -566,7 +566,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public void testGlobalFailover_readyStopWorkflow_withKilledTasks() {
final String yaml = "/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.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()));
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java
index 5154d698ca..2fd2797ddb 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java
+++ b/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 org.awaitility.Awaitility.await;
+import org.apache.dolphinscheduler.common.enums.Flag;
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.extract.master.command.RunWorkflowCommandParam;
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 java.time.Duration;
+import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.DisplayName;
@@ -60,7 +63,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_oneSuccessTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_success.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()
.workflowDefinition(workflow)
@@ -111,7 +114,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_oneFailedTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_failed.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()
.workflowDefinition(workflow)
@@ -162,7 +165,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public void testPauseWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/pause/workflow_with_three_parallel_three_fake_task_success.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()
.workflowDefinition(workflow)
@@ -236,4 +239,82 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
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 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();
+ }
+
}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverPauseIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverPauseIT.java
new file mode 100644
index 0000000000..393ea84b02
--- /dev/null
+++ b/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 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();
+ }
+
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverStopIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverStopIT.java
new file mode 100644
index 0000000000..e676325959
--- /dev/null
+++ b/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 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();
+ }
+
+}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java
index bd9732345d..d80fa85bc1 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java
+++ b/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 org.awaitility.Awaitility.await;
+import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
@@ -62,7 +63,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_oneSuccessTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_success.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()
.workflowDefinition(workflow)
@@ -111,7 +112,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_oneFailedTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_failed.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()
.workflowDefinition(workflow)
@@ -160,7 +161,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public void testStopWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/stop/workflow_with_three_parallel_three_fake_task_success.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()
.workflowDefinition(workflow)
@@ -208,4 +209,76 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
});
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();
+ }
}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java
index 833d0d3718..3ea319a41b 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java
+++ b/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() {
final String yaml = "/it/scheduling/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
- final WorkflowDefinition workflow = context.getWorkflow();
+ final WorkflowDefinition workflow = context.getWorkflows().get(0);
final Schedule schedule = Schedule.builder()
.processDefinitionCode(workflow.getCode())
@@ -89,7 +89,7 @@ public class WorkflowSchedulingIT extends AbstractMasterIntegrationTest {
WorkflowOperator.WorkflowSchedulingDTO workflowSchedulingDTO = WorkflowOperator.WorkflowSchedulingDTO.builder()
.project(context.getProject())
- .workflow(context.getWorkflow())
+ .workflow(context.getWorkflows().get(0))
.schedule(schedule)
.build();
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java
index 28d34e63f5..039b2f4ced 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java
+++ b/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.dao.entity.TaskInstance;
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.DataType;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
@@ -72,21 +73,21 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneSuccessTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_success.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()
.workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam())
.build();
- workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
+ final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
Assertions
- .assertThat(repository.queryWorkflowInstance(workflow))
- .satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState())
- .isEqualTo(WorkflowExecutionStatus.SUCCESS));
+ .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
+ .matches(
+ workflowInstance -> workflowInstance.getState() == WorkflowExecutionStatus.SUCCESS);
Assertions
.assertThat(repository.queryTaskInstance(workflow))
.satisfiesExactly(taskInstance -> {
@@ -98,12 +99,116 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
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 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 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
@DisplayName("Test start a workflow which using workflow params")
public void testStartWorkflow_usingWorkflowParam() {
final String yaml = "/it/start/workflow_with_global_param.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()
.workflowDefinition(workflow)
@@ -139,7 +244,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_usingCommandParam() {
final String yaml = "/it/start/workflow_with_global_param.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
- final WorkflowDefinition workflow = context.getWorkflow();
+ final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
.commandParams(Lists.newArrayList(Property.builder()
@@ -184,7 +289,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneFailedTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_failed.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()
.workflowDefinition(workflow)
@@ -215,7 +320,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_oneFailedTaskWithRetry() {
final String yaml = "/it/start/workflow_with_one_fake_task_failed_with_retry.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()
.workflowDefinition(workflow)
@@ -268,7 +373,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoSerialSuccessTask() {
String yaml = "/it/start/workflow_with_two_serial_fake_task_success.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()
.workflowDefinition(workflow)
@@ -307,7 +412,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoSerialFailedTask() {
final String yaml = "/it/start/workflow_with_two_serial_fake_task_failed.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()
.workflowDefinition(workflow)
@@ -338,7 +443,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_two_parallel_fake_task_success.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()
.workflowDefinition(workflow)
@@ -376,7 +481,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_twoParallelFailedTask() {
final String yaml = "/it/start/workflow_with_two_parallel_fake_task_failed.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()
.workflowDefinition(workflow)
@@ -415,7 +520,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflow_with_threeParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.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()
.workflowDefinition(workflow)
@@ -482,7 +587,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public void testStartWorkflowFromStartNodes_with_threeParallelSuccessTask() {
final String yaml = "/it/start/workflow_with_three_parallel_three_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
- final WorkflowDefinition workflow = context.getWorkflow();
+ final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
.startNodes(Lists.newArrayList(6L))
diff --git a/dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml
index 17d8d34aaa..7b6f64ff04 100644
--- a/dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml
index a5f347c2fe..618fa44733 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml
index 40896794cd..260042052c 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml
index d8a4ccf62d..51eb99b9ed 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml
index b1e1a6a16e..9cbd00d7a2 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml
index dc6fa7f242..b7efaee10e 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml
index fac3cf3a8c..4b3ca71bd4 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml
index 1d0810469f..11a3574863 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml
index 929d2319df..eeb30b59f3 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml
index 65097b00ea..28697650a7 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml
index d4a32dee40..8d4975ec42 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml
index 145525a982..609d960d5d 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml
index 33cc9627c3..82c14b1cf8 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml
index 61b2bc33ba..a06410be3e 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml
index db615c9a20..da76f9d7ab 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml
index 6e2d31c923..2fbbe53167 100644
--- a/dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml
index 22e0d24ab6..57aeedc54c 100644
--- a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml
index 17d8d34aaa..7b6f64ff04 100644
--- a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_sub_workflow_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_sub_workflow_task_success.yaml
new file mode 100644
index 0000000000..2192122a8a
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml
index fe5339c7c3..941b012148 100644
--- a/dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_three_parallel_three_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel success tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_three_parallel_three_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel success tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A1
diff --git a/dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml b/dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml
index 8cb566fc4f..b5ff2db7ae 100644
--- a/dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml
+++ b/dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml
@@ -73,17 +73,17 @@ taskInstances:
varPool: '[]'
taskExecuteType: BATCH
-workflow:
- name: workflow_with_two_serial_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two serial tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_serial_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two serial tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/recover_paused/workflow_with_sub_workflow_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/recover_paused/workflow_with_sub_workflow_task_success.yaml
new file mode 100644
index 0000000000..2192122a8a
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/recover_stopped/workflow_with_sub_workflow_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/recover_stopped/workflow_with_sub_workflow_task_success.yaml
new file mode 100644
index 0000000000..2192122a8a
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml
index aa49cc572a..4de8c8c342 100644
--- a/dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml
index 20541fa708..c39e90896b 100644
--- a/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
workflowInstance:
id: 1
diff --git a/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml b/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml
index 5d13b080dc..527ecf40b3 100644
--- a/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml
+++ b/dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml
@@ -74,17 +74,17 @@ taskInstances:
varPool: '[]'
taskExecuteType: BATCH
-workflow:
- name: workflow_with_two_parallel_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel success tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_parallel_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel success tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml
index 17d8d34aaa..7b6f64ff04 100644
--- a/dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml
index a82ebfb37a..522334368c 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml
+++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml
@@ -24,18 +24,18 @@ project:
createTime: 2024-08-12 00:00:00
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- globalParams: '[{"prop":"name","value":"workflowParam","direct":"IN","type":"VARCHAR"}]'
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ globalParams: '[{"prop":"name","value":"workflowParam","direct":"IN","type":"VARCHAR"}]'
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml
index 22e0d24ab6..57aeedc54c 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml
index 649f98d315..1564c96c2c 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml
index 17d8d34aaa..7b6f64ff04 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_failed.yaml
new file mode 100644
index 0000000000..09d5d6f11e
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_sub_workflow_task_success.yaml
new file mode 100644
index 0000000000..19920c0d83
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml
index fe5339c7c3..941b012148 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_three_parallel_three_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel success tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_three_parallel_three_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel success tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A1
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml
index 40d34c44d7..b0b4c7e647 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_two_parallel_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel failed tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_parallel_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel failed tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml
index 92c2e380ee..56f8642b75 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_two_parallel_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel success tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_parallel_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel success tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml
index 2b22fea457..7b52c3c005 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_two_serial_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two serial tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_serial_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two serial tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml
index d981796fe2..d5b09e498e 100644
--- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_two_serial_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two serial tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_two_serial_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two serial tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml b/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml
index 22e0d24ab6..57aeedc54c 100644
--- a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_failed
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_failed
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml
index 17d8d34aaa..7b6f64ff04 100644
--- a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_one_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with single task
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_one_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with single task
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A
diff --git a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_sub_workflow_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_sub_workflow_task_success.yaml
new file mode 100644
index 0000000000..2192122a8a
--- /dev/null
+++ b/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
diff --git a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml b/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml
index fe5339c7c3..941b012148 100644
--- a/dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml
+++ b/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
updateTime: 2021-08-12 00:00:00
-workflow:
- name: workflow_with_three_parallel_three_fake_task_success
- code: 1
- version: 1
- projectCode: 1
- description: This is a fake workflow with two parallel success tasks
- releaseState: ONLINE
- createTime: 2024-08-12 00:00:00
- updateTime: 2021-08-12 00:00:00
- userId: 1
- executionType: PARALLEL
+workflows:
+ - name: workflow_with_three_parallel_three_fake_task_success
+ code: 1
+ version: 1
+ projectCode: 1
+ description: This is a fake workflow with two parallel success tasks
+ releaseState: ONLINE
+ createTime: 2024-08-12 00:00:00
+ updateTime: 2021-08-12 00:00:00
+ userId: 1
+ executionType: PARALLEL
tasks:
- name: A1