This is an automated email from the ASF dual-hosted git repository.

wenjun pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git


The following commit(s) were added to refs/heads/dev by this push:
     new 879e966576 [Fix-16990] Fix WorkflowExecutionGraph#isTaskFinish is not 
working correctly (#16995)
879e966576 is described below

commit 879e9665764567391facb4c491654a490a3108a3
Author: Wenjun Ruan <[email protected]>
AuthorDate: Fri Feb 7 10:42:03 2025 +0800

    [Fix-16990] Fix WorkflowExecutionGraph#isTaskFinish is not working 
correctly (#16995)
---
 .../engine/graph/IWorkflowExecutionGraph.java      |  17 +++
 .../engine/graph/WorkflowExecutionGraph.java       |  48 ++++----
 .../statemachine/AbstractWorkflowStateAction.java  |   4 +-
 .../WorkflowReadyPauseStateAction.java             |   2 +-
 .../statemachine/WorkflowReadyStopStateAction.java |   2 +-
 .../statemachine/WorkflowRunningStateAction.java   |   2 +-
 .../integration/cases/WorkflowStartTestCase.java   |  84 +++++++++++++
 ...ake_task_with_multiple_predecessors_failed.yaml | 132 +++++++++++++++++++++
 ...ke_task_with_multiple_predecessors_success.yaml | 132 +++++++++++++++++++++
 9 files changed, 396 insertions(+), 27 deletions(-)

diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/IWorkflowExecutionGraph.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/IWorkflowExecutionGraph.java
index 3f168cce6e..1c3af50915 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/IWorkflowExecutionGraph.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/IWorkflowExecutionGraph.java
@@ -80,11 +80,27 @@ public interface IWorkflowExecutionGraph {
      */
     boolean isTaskExecutionRunnableActive(final ITaskExecutionRunnable 
taskExecutionRunnable);
 
+    /**
+     * Whether the given task is inactive.
+     * <p> A task is inactive means the task has been `executed`.
+     */
+    boolean isTaskExecutionRunnableInActive(final ITaskExecutionRunnable 
taskExecutionRunnable);
+
     /**
      * Whether the given task is killed.
      */
     boolean isTaskExecutionRunnableKilled(final ITaskExecutionRunnable 
taskExecutionRunnable);
 
+    /**
+     * Whether the given task is failure.
+     */
+    boolean isTaskExecutionRunnableFailed(final ITaskExecutionRunnable 
taskExecutionRunnable);
+
+    /**
+     * Whether the given task is paused.
+     */
+    boolean isTaskExecutionRunnablePaused(final ITaskExecutionRunnable 
taskExecutionRunnable);
+
     /**
      * Get the active TaskExecutionRunnable list.
      * <p> The active TaskExecutionRunnable means the task is handling in the 
workflow execution graph.
@@ -176,6 +192,7 @@ public interface IWorkflowExecutionGraph {
      * Check whether the given task is the end of the task chain.
      * <p> If the given task has no successor, then it is the end of the task 
chain.
      * <p> If the given task is killed or paused, then it is the end of the 
task chain.
+     * <p> If the given task is failure, and all its successors are condition 
task then it is not end of a task chain.
      */
     boolean isEndOfTaskChain(final ITaskExecutionRunnable 
taskExecutionRunnable);
 
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowExecutionGraph.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowExecutionGraph.java
index dd89debe6c..2439d9f840 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowExecutionGraph.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/WorkflowExecutionGraph.java
@@ -51,6 +51,8 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
 
     private final Set<String> activeTaskExecutionRunnable;
 
+    private final Set<String> inActiveTaskExecutionRunnable;
+
     public WorkflowExecutionGraph() {
         this.failureTaskChains = new HashSet<>();
         this.pausedTaskChains = new HashSet<>();
@@ -60,6 +62,7 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
         this.successors = new HashMap<>();
         this.totalTaskExecuteRunnableMap = new HashMap<>();
         this.activeTaskExecutionRunnable = new HashSet<>();
+        this.inActiveTaskExecutionRunnable = new HashSet<>();
     }
 
     @Override
@@ -143,11 +146,26 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
         return 
activeTaskExecutionRunnable.contains(taskExecutionRunnable.getName());
     }
 
+    @Override
+    public boolean isTaskExecutionRunnableInActive(ITaskExecutionRunnable 
taskExecutionRunnable) {
+        return 
inActiveTaskExecutionRunnable.contains(taskExecutionRunnable.getName());
+    }
+
     @Override
     public boolean isTaskExecutionRunnableKilled(final ITaskExecutionRunnable 
taskExecutionRunnable) {
         return killedTaskChains.contains(taskExecutionRunnable.getName());
     }
 
+    @Override
+    public boolean isTaskExecutionRunnableFailed(ITaskExecutionRunnable 
taskExecutionRunnable) {
+        return failureTaskChains.contains(taskExecutionRunnable.getName());
+    }
+
+    @Override
+    public boolean isTaskExecutionRunnablePaused(ITaskExecutionRunnable 
taskExecutionRunnable) {
+        return pausedTaskChains.contains(taskExecutionRunnable.getName());
+    }
+
     @Override
     public List<ITaskExecutionRunnable> getActiveTaskExecutionRunnable() {
         return activeTaskExecutionRunnable
@@ -165,10 +183,10 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
     public boolean isTriggerConditionMet(final ITaskExecutionRunnable 
taskExecutionRunnable) {
         return getPredecessors(taskExecutionRunnable.getName())
                 .stream()
-                .allMatch(predecessor -> isTaskFinish(predecessor)
-                        && !isTaskFailure(predecessor)
-                        && !isTaskPaused(predecessor)
-                        && !isTaskKilled(predecessor));
+                .allMatch(predecessor -> 
isTaskExecutionRunnableInActive(predecessor)
+                        && !isTaskExecutionRunnableFailed(predecessor)
+                        && !isTaskExecutionRunnablePaused(predecessor)
+                        && !isTaskExecutionRunnableKilled(predecessor));
     }
 
     @Override
@@ -209,6 +227,7 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
     @Override
     public void markTaskExecutionRunnableInActive(final ITaskExecutionRunnable 
taskExecutionRunnable) {
         activeTaskExecutionRunnable.remove(taskExecutionRunnable.getName());
+        inActiveTaskExecutionRunnable.add(taskExecutionRunnable.getName());
     }
 
     @Override
@@ -242,8 +261,9 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
     @Override
     public boolean isEndOfTaskChain(final ITaskExecutionRunnable 
taskExecutionRunnable) {
         return successors.get(taskExecutionRunnable.getName()).isEmpty()
-                || killedTaskChains.contains(taskExecutionRunnable.getName())
-                || pausedTaskChains.contains(taskExecutionRunnable.getName());
+                || isTaskExecutionRunnableKilled(taskExecutionRunnable)
+                || isTaskExecutionRunnablePaused(taskExecutionRunnable)
+                || isTaskExecutionRunnableFailed(taskExecutionRunnable);
     }
 
     @Override
@@ -291,22 +311,6 @@ public class WorkflowExecutionGraph implements 
IWorkflowExecutionGraph {
                         || 
TaskTypeUtils.isConditionTask(taskExecutionRunnable.getTaskInstance().getTaskType()));
     }
 
-    private boolean isTaskFinish(final ITaskExecutionRunnable 
taskExecutionRunnable) {
-        return 
!activeTaskExecutionRunnable.contains(taskExecutionRunnable.getName());
-    }
-
-    private boolean isTaskFailure(final ITaskExecutionRunnable 
taskExecutionRunnable) {
-        return failureTaskChains.contains(taskExecutionRunnable.getName());
-    }
-
-    private boolean isTaskPaused(final ITaskExecutionRunnable 
taskExecutionRunnable) {
-        return pausedTaskChains.contains(taskExecutionRunnable.getName());
-    }
-
-    private boolean isTaskKilled(final ITaskExecutionRunnable 
taskExecutionRunnable) {
-        return killedTaskChains.contains(taskExecutionRunnable.getName());
-    }
-
     private void assertTaskExecutionRunnableState(final ITaskExecutionRunnable 
taskExecutionRunnable,
                                                   final TaskExecutionStatus 
taskExecutionStatus) {
         final TaskInstance taskInstance = 
taskExecutionRunnable.getTaskInstance();
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/AbstractWorkflowStateAction.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/AbstractWorkflowStateAction.java
index 1cd6ad21b7..4e05255d64 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/AbstractWorkflowStateAction.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/AbstractWorkflowStateAction.java
@@ -115,8 +115,8 @@ public abstract class AbstractWorkflowStateAction 
implements IWorkflowStateActio
         }
     }
 
-    protected void onTaskFinish(final IWorkflowExecutionRunnable 
workflowExecutionRunnable,
-                                final ITaskExecutionRunnable 
taskExecutionRunnable) {
+    protected void tryToTriggerSuccessorsAfterTaskFinish(final 
IWorkflowExecutionRunnable workflowExecutionRunnable,
+                                                         final 
ITaskExecutionRunnable taskExecutionRunnable) {
         final IWorkflowExecutionGraph workflowExecutionGraph = 
workflowExecutionRunnable.getWorkflowExecutionGraph();
         if (workflowExecutionGraph.isEndOfTaskChain(taskExecutionRunnable)) {
             emitWorkflowFinishedEventIfApplicable(workflowExecutionRunnable);
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyPauseStateAction.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyPauseStateAction.java
index 1d1925f272..cb48b33acd 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyPauseStateAction.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyPauseStateAction.java
@@ -52,7 +52,7 @@ public class WorkflowReadyPauseStateAction extends 
AbstractWorkflowStateAction {
     public void topologyLogicalTransitionEventAction(final 
IWorkflowExecutionRunnable workflowExecutionRunnable,
                                                      final 
WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent 
workflowTopologyLogicalTransitionWithTaskFinishEvent) {
         throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
-        super.onTaskFinish(workflowExecutionRunnable,
+        super.tryToTriggerSuccessorsAfterTaskFinish(workflowExecutionRunnable,
                 
workflowTopologyLogicalTransitionWithTaskFinishEvent.getTaskExecutionRunnable());
     }
 
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyStopStateAction.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyStopStateAction.java
index 8b1f393ffe..c5c94c18c5 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyStopStateAction.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowReadyStopStateAction.java
@@ -52,7 +52,7 @@ public class WorkflowReadyStopStateAction extends 
AbstractWorkflowStateAction {
     public void topologyLogicalTransitionEventAction(final 
IWorkflowExecutionRunnable workflowExecutionRunnable,
                                                      final 
WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent 
workflowTopologyLogicalTransitionWithTaskFinishEvent) {
         throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
-        super.onTaskFinish(workflowExecutionRunnable,
+        super.tryToTriggerSuccessorsAfterTaskFinish(workflowExecutionRunnable,
                 
workflowTopologyLogicalTransitionWithTaskFinishEvent.getTaskExecutionRunnable());
     }
 
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowRunningStateAction.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowRunningStateAction.java
index 2ef810aba4..15e360cdb6 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowRunningStateAction.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowRunningStateAction.java
@@ -53,7 +53,7 @@ public class WorkflowRunningStateAction extends 
AbstractWorkflowStateAction {
                                                      final 
IWorkflowExecutionRunnable workflowExecutionRunnable,
                                                      final 
WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent 
workflowTopologyLogicalTransitionWithTaskFinishEvent) {
         throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
-        super.onTaskFinish(workflowExecutionRunnable,
+        super.tryToTriggerSuccessorsAfterTaskFinish(workflowExecutionRunnable,
                 
workflowTopologyLogicalTransitionWithTaskFinishEvent.getTaskExecutionRunnable());
     }
 
diff --git 
a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java
 
b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java
index 55558eece2..1041a34a52 100644
--- 
a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java
+++ 
b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java
@@ -252,6 +252,90 @@ public class WorkflowStartTestCase extends 
AbstractMasterIntegrationTestCase {
         masterContainer.assertAllResourceReleased();
     }
 
+    @Test
+    @DisplayName("Test start a workflow with one fake task(A) with multiple 
predecessors run success")
+    void testStartWorkflow_with_oneTaskWithMultiplePredecessors_runSuccess() {
+        final String yaml = 
"/it/start/workflow_with_one_fake_task_with_multiple_predecessors_success.yaml";
+        final WorkflowTestCaseContext context = 
workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
+        final WorkflowDefinition parentWorkflow = context.getOneWorkflow();
+
+        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);
+
+                    Assertions
+                            
.assertThat(repository.queryTaskInstance(workflowInstanceId))
+                            .hasSize(4)
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("A");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            })
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("B");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            })
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("C");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            })
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("D");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            });
+                });
+        masterContainer.assertAllResourceReleased();
+    }
+
+    @Test
+    @DisplayName("Test start a workflow with one fake task(A) with multiple 
predecessors run failed")
+    void testStartWorkflow_with_oneTaskWithMultiplePredecessors_runFailed() {
+        final String yaml = 
"/it/start/workflow_with_one_fake_task_with_multiple_predecessors_failed.yaml";
+        final WorkflowTestCaseContext context = 
workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
+        final WorkflowDefinition parentWorkflow = context.getOneWorkflow();
+
+        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);
+
+                    Assertions
+                            
.assertThat(repository.queryTaskInstance(workflowInstanceId))
+                            .hasSize(3)
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("A");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            })
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("B");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE);
+                            })
+                            .anySatisfy(taskInstance -> {
+                                
assertThat(taskInstance.getName()).isEqualTo("C");
+                                
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
+                            });
+                });
+        masterContainer.assertAllResourceReleased();
+    }
+
     @Test
     @DisplayName("Test start a workflow with one sub workflow task(A) failed")
     public void testStartWorkflow_with_subWorkflowTask_failed() {
diff --git 
a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_with_multiple_predecessors_failed.yaml
 
b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_with_multiple_predecessors_failed.yaml
new file mode 100644
index 0000000000..075a45da33
--- /dev/null
+++ 
b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_with_multiple_predecessors_failed.yaml
@@ -0,0 +1,132 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# A(success) -> B(failed) -> D(success)
+# C(success) -> D(success)
+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: workflow_with_one_fake_task_with_multiple_predecessors_failed
+    code: 1
+    version: 1
+    projectCode: 1
+    description: This is a fake workflow with one task which has multiple 
predecessors
+    releaseState: ONLINE
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2021-08-12 00:00:00
+    userId: 1
+    executionType: PARALLEL
+
+tasks:
+  - name: A
+    code: 1
+    version: 1
+    projectCode: 1
+    userId: 1
+    taskType: LogicFakeTask
+    taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 2"}'
+    workerGroup: default
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2021-08-12 00:00:00
+    taskExecuteType: BATCH
+  - name: B
+    code: 2
+    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
+  - name: C
+    code: 3
+    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
+  - name: D
+    code: 4
+    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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 1
+    preTaskVersion: 1
+    postTaskCode: 2
+    postTaskVersion: 1
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2024-08-12 00:00:00
+  - projectCode: 1
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 2
+    preTaskVersion: 1
+    postTaskCode: 4
+    postTaskVersion: 1
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2024-08-12 00:00:00
+  - projectCode: 1
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 3
+    preTaskVersion: 1
+    postTaskCode: 4
+    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_one_fake_task_with_multiple_predecessors_success.yaml
 
b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_with_multiple_predecessors_success.yaml
new file mode 100644
index 0000000000..53cc326986
--- /dev/null
+++ 
b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_with_multiple_predecessors_success.yaml
@@ -0,0 +1,132 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# A(success) -> B(success) -> D(success)
+# C(success) -> D(success)
+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: workflow_with_one_fake_task_with_multiple_predecessors_success
+    code: 1
+    version: 1
+    projectCode: 1
+    description: This is a fake workflow with one task which has multiple 
predecessors
+    releaseState: ONLINE
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2021-08-12 00:00:00
+    userId: 1
+    executionType: PARALLEL
+
+tasks:
+  - name: A
+    code: 1
+    version: 1
+    projectCode: 1
+    userId: 1
+    taskType: LogicFakeTask
+    taskParams: '{"localParams":null,"varPool":[],"shellScript":"sleep 2"}'
+    workerGroup: default
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2021-08-12 00:00:00
+    taskExecuteType: BATCH
+  - name: B
+    code: 2
+    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
+  - name: C
+    code: 3
+    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
+  - name: D
+    code: 4
+    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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 1
+    preTaskVersion: 1
+    postTaskCode: 2
+    postTaskVersion: 1
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2024-08-12 00:00:00
+  - projectCode: 1
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 2
+    preTaskVersion: 1
+    postTaskCode: 4
+    postTaskVersion: 1
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2024-08-12 00:00:00
+  - projectCode: 1
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 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
+    workflowDefinitionCode: 1
+    workflowDefinitionVersion: 1
+    preTaskCode: 3
+    preTaskVersion: 1
+    postTaskCode: 4
+    postTaskVersion: 1
+    createTime: 2024-08-12 00:00:00
+    updateTime: 2024-08-12 00:00:00

Reply via email to