diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java index 5ddcf13c8917..49abdc0e1cbd 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java @@ -29,6 +29,10 @@ public enum TaskLifecycleEventType implements ILifecycleEventType { * Dispatch the task instance to target. */ DISPATCH, + /** + * Severe and unrecoverable errors, such as initialization failure. + */ + FATAL, /** * The task instance is dispatched to the target executor server. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java new file mode 100644 index 000000000000..88cfc9a478f6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskFatalLifecycleEvent.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Date; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +@AllArgsConstructor +public class TaskFatalLifecycleEvent extends AbstractTaskLifecycleEvent { + + private final ITaskExecutionRunnable taskExecutionRunnable; + + private final Date endTime; + + @Override + public ILifecycleEventType getEventType() { + return TaskLifecycleEventType.FATAL; + } + + @Override + public String toString() { + return "TaskFatalLifecycleEvent{" + + "task=" + taskExecutionRunnable.getName() + + ", endTime=" + endTime + + '}'; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java new file mode 100644 index 000000000000..8277d8b2caa7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFatalLifecycleEventHandler.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; +import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; + +import org.springframework.stereotype.Component; + +@Component +public class TaskFatalLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + + @Override + public void handle(final ITaskStateAction taskStateAction, + final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + taskStateAction.onFatalEvent(workflowExecutionRunnable, taskExecutionRunnable, taskFatalEvent); + } + + @Override + public ILifecycleEventType matchEventType() { + return TaskLifecycleEventType.FATAL; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java index ad0e652d4866..414f3bb40122 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java @@ -36,6 +36,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; @@ -99,6 +100,29 @@ protected void releaseTaskInstanceResourcesIfNeeded(final ITaskExecutionRunnable } } + @Override + public void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent) { + releaseTaskInstanceResourcesIfNeeded(taskExecutionRunnable); + + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + taskInstance.setState(TaskExecutionStatus.FAILURE); + taskInstance.setEndTime(taskFatalEvent.getEndTime()); + taskInstanceDao.updateById(taskInstance); + + // If all successors are condition tasks, then the task will not be marked as failure. + // And the DAG will continue to execute. + final IWorkflowExecutionGraph workflowExecutionGraph = taskExecutionRunnable.getWorkflowExecutionGraph(); + if (workflowExecutionGraph.isAllSuccessorsAreConditionTask(taskExecutionRunnable)) { + mergeTaskVarPoolToWorkflow(workflowExecutionRunnable, taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + return; + } + taskExecutionRunnable.getWorkflowExecutionGraph().markTaskExecutionRunnableChainFailure(taskExecutionRunnable); + publishWorkflowInstanceTopologyLogicalTransitionEvent(taskExecutionRunnable); + } + @Override public void onDispatchedEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java index a041de5e3ca6..44281e7129e8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailoverLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; @@ -91,6 +92,14 @@ void onDispatchEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskDispatchLifecycleEvent taskDispatchEvent); + /** + * Perform the necessary actions when the task in a certain state receive a {@link TaskFatalLifecycleEvent}. + *

This method is called when the task encounters an unrecoverable error (e.g., initialization failure). + */ + void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskFatalLifecycleEvent taskFatalEvent); + /** * Perform the necessary actions when the task in a certain state receive a {@link TaskDispatchedLifecycleEvent}. *

This method is called when the task has been dispatched to executor. diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java index 327cd942582d..814262676c81 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java @@ -26,6 +26,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailoverLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; @@ -37,6 +38,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import java.util.Date; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; @@ -109,7 +111,17 @@ public void onDispatchEvent(final IWorkflowExecutionRunnable workflowExecutionRu taskInstance.getDelayTime(), remainTimeMills); } - taskExecutionRunnable.initializeTaskExecutionContext(); + try { + taskExecutionRunnable.initializeTaskExecutionContext(); + } catch (Exception ex) { + log.error("Current taskInstance: {} initializeTaskExecutionContext error", taskInstance.getName(), ex); + final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent); + return; + } workerGroupDispatcherCoordinator.dispatchTask(taskExecutionRunnable, remainTimeMills); } 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 a2f0331f3483..fe5a497aa64a 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 @@ -849,6 +849,36 @@ public void testStartWorkflow_with_oneFailedTask() { masterContainer.assertAllResourceReleased(); } + @Test + @DisplayName("Test start a workflow with one fake task(A) fatal") + public void testStartWorkflow_with_oneFatalTask() { + final String yaml = "/it/start/workflow_with_one_fake_task_fatal.yaml"; + final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml); + final WorkflowDefinition workflow = context.getOneWorkflow(); + + final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder() + .workflowDefinition(workflow) + .runWorkflowCommandParam(new RunWorkflowCommandParam()) + .build(); + workflowOperator.manualTriggerWorkflow(workflowTriggerDTO); + + await() + .atMost(Duration.ofMinutes(1)) + .untilAsserted(() -> { + Assertions + .assertThat(repository.queryWorkflowInstance(workflow)) + .satisfiesExactly(workflowInstance -> assertThat(workflowInstance.getState()) + .isEqualTo(WorkflowExecutionStatus.FAILURE)); + Assertions + .assertThat(repository.queryTaskInstance(workflow)) + .satisfiesExactly(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("A"); + assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE); + }); + }); + masterContainer.assertAllResourceReleased(); + } + @Test @DisplayName("Test start a workflow with one fake task(A) failed") public void testStartWorkflow_with_oneFailedTaskWithRetry() { @@ -1403,6 +1433,46 @@ void testStartWorkflow_with_oneConditionTaskWithOneFakePredecessor_runFailed() { masterContainer.assertAllResourceReleased(); } + @Test + @DisplayName("Test start a workflow with one condition task(B) when one fake predecessor task(A) run fatal") + void testStartWorkflow_with_oneConditionTaskWithOneFakePredecessor_runFatal() { + final String yaml = "/it/start/workflow_with_one_condition_task_with_one_fake_predecessor_fatal.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(3) + .anySatisfy(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("A"); + assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE); + }) + .anySatisfy(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("B"); + 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 condition task(B) which is forbidden when one fake predecessor task(A) run failed") void testStartWorkflow_with_oneForbiddenConditionTaskWithOneFakePredecessor_runFailed() { @@ -1435,4 +1505,37 @@ void testStartWorkflow_with_oneForbiddenConditionTaskWithOneFakePredecessor_runF }); masterContainer.assertAllResourceReleased(); } + + @Test + @DisplayName("Test start a workflow with one condition task(B) which is forbidden when one fake predecessor task(A) run fatal") + void testStartWorkflow_with_oneForbiddenConditionTaskWithOneFakePredecessor_runFatal() { + final String yaml = + "/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_fatal.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(1) + .satisfiesExactly(taskInstance -> { + assertThat(taskInstance.getName()).isEqualTo("A"); + assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE); + }); + }); + masterContainer.assertAllResourceReleased(); + } } diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_condition_task_with_one_fake_predecessor_fatal.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_condition_task_with_one_fake_predecessor_fatal.yaml new file mode 100644 index 000000000000..748f3634bf2e --- /dev/null +++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_condition_task_with_one_fake_predecessor_fatal.yaml @@ -0,0 +1,123 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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(fatal-failed) -> B(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: 2024-08-12 00:00:00 + +workflows: + - name: workflow_with_one_condition_task_with_one_fake_predecessor_fatal + code: 1 + version: 1 + projectCode: 1 + description: This is a fake workflow with one condition task which has one predecessor fatal + releaseState: ONLINE + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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":"echo success"}' + workerGroup: default + environmentCode : 144873539254368 + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + taskExecuteType: BATCH + - name: B + code: 2 + version: 1 + projectCode: 1 + userId: 1 + taskType: CONDITIONS + taskParams: '{"localParams":[],"resourceList":[],"dependence":{"relation":"AND","dependTaskList":[{"relation":"AND","dependItemList":[{"depTaskCode":1,"status":"SUCCESS"}]}]},"conditionResult":{"successNode":[3],"failedNode":[4]}},' + workerGroup: default + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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: 2024-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 failed"}' + workerGroup: default + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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: 3 + 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 diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_fatal.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_fatal.yaml new file mode 100644 index 000000000000..bbd5adf84876 --- /dev/null +++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_fatal.yaml @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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: 2024-08-12 00:00:00 + +workflows: + - name: workflow_with_one_fake_task_fatal + 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: 2024-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 5 && echo success"}' + workerGroup: default + environmentCode: 144873539254368 + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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 diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_failed.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_failed.yaml index c312175c38a2..bd5a9d360ec3 100644 --- a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_failed.yaml +++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_failed.yaml @@ -27,11 +27,11 @@ project: updateTime: 2021-08-12 00:00:00 workflows: - - name: workflow_with_one_condition_task_with_one_fake_predecessor_failed + - name: workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_failed code: 1 version: 1 projectCode: 1 - description: This is a fake workflow with one condition task which has one predecessor failed + description: This is a fake workflow with one condition task which is forbidden and has one predecessor failed releaseState: ONLINE createTime: 2024-08-12 00:00:00 updateTime: 2021-08-12 00:00:00 diff --git a/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_fatal.yaml b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_fatal.yaml new file mode 100644 index 000000000000..63039224c4a4 --- /dev/null +++ b/dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_fatal.yaml @@ -0,0 +1,125 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A(failed) -> B(Condition)(forbidden) -> C(success) +# -> D(failed) +project: + name: MasterIntegrationTest + code: 1 + description: This is a fake project + userId: 1 + userName: admin + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + +workflows: + - name: workflow_with_one_forbidden_condition_task_with_one_fake_predecessor_fatal + code: 1 + version: 1 + projectCode: 1 + description: This is a fake workflow with one forbidden condition task which has one predecessor fatal + releaseState: ONLINE + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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":"echo success"}' + workerGroup: default + environmentCode: 144873539254368 + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + taskExecuteType: BATCH + - name: B + code: 2 + version: 1 + projectCode: 1 + userId: 1 + taskType: CONDITIONS + taskParams: '{"localParams":[],"resourceList":[],"dependence":{"relation":"AND","dependTaskList":[{"relation":"AND","dependItemList":[{"depTaskCode":1,"status":"SUCCESS"}]}]},"conditionResult":{"successNode":[3],"failedNode":[4]}},' + workerGroup: default + createTime: 2024-08-12 00:00:00 + updateTime: 2024-08-12 00:00:00 + taskExecuteType: BATCH + flag: NO + - 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: 2024-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 failed"}' + workerGroup: default + createTime: 2024-08-12 00:00:00 + updateTime: 2024-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: 3 + 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