Browse Source

[Fix-16338][Test] Fix Master integration test case has been skipped in ci (#16637)

dev
Wenjun Ruan 2 months ago committed by GitHub
parent
commit
3c944bccee
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 1
      dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertServer.java
  2. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java
  3. 5
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/lifecycle/ServerLifeCycleManager.java
  4. 6
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/BaseHeartBeatTask.java
  5. 11
      dolphinscheduler-master/pom.xml
  6. 13
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  7. 16
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java
  8. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java
  9. 13
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/CommandEngine.java
  10. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/AbstractCommandHandler.java
  11. 13
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/SystemEventBusFireWorker.java
  12. 1
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java
  13. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/listener/WorkflowSuccessLifecycleListener.java
  14. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java
  15. 17
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/AbstractMasterIntegrationTestCase.java
  16. 2
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/Repository.java
  17. 6
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowOperator.java
  18. 4
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowTestCaseContext.java
  19. 26
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowTestCaseContextFactory.java
  20. 6
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/YamlFactory.java
  21. 22
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowBackfillTestCase.java
  22. 58
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceFailoverTestCase.java
  23. 24
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstancePauseTestCase.java
  24. 18
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverFailureTaskTestCase.java
  25. 18
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverPauseTestCase.java
  26. 18
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverStopTestCase.java
  27. 22
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRepeatRunningTestCase.java
  28. 24
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceStopTestCase.java
  29. 18
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowSchedulingTestCase.java
  30. 42
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java
  31. 6
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java
  32. 13
      dolphinscheduler-master/src/test/resources/application.yaml
  33. 4
      dolphinscheduler-master/src/test/resources/it/backfill/workflow_with_one_fake_task_success.yaml
  34. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_dispatched_fake_task.yaml
  35. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_failed_fake_task.yaml
  36. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_paused_fake_task.yaml
  37. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_submitted_fake_task.yaml
  38. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyPause_workflowInstance_with_one_success_fake_task.yaml
  39. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_dispatched_fake_task.yaml
  40. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_failed_fake_task.yaml
  41. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_killed_fake_task.yaml
  42. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_submitted_fake_task.yaml
  43. 14
      dolphinscheduler-master/src/test/resources/it/failover/readyStop_workflowInstance_with_one_success_fake_task.yaml
  44. 14
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_dispatched_fake_task.yaml
  45. 14
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_failed_fake_task.yaml
  46. 14
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_running_fake_task.yaml
  47. 14
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_submitted_fake_task.yaml
  48. 14
      dolphinscheduler-master/src/test/resources/it/failover/running_workflowInstance_with_one_success_fake_task.yaml
  49. 4
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_failed.yaml
  50. 4
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_one_fake_task_success.yaml
  51. 36
      dolphinscheduler-master/src/test/resources/it/pause/workflow_with_three_parallel_three_fake_task_success.yaml
  52. 18
      dolphinscheduler-master/src/test/resources/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml
  53. 14
      dolphinscheduler-master/src/test/resources/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml
  54. 14
      dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_one_fake_task_success.yaml
  55. 18
      dolphinscheduler-master/src/test/resources/it/repeat_running/success_workflow_with_task_only.yaml
  56. 4
      dolphinscheduler-master/src/test/resources/it/scheduling/workflow_with_one_fake_task_success.yaml
  57. 8
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_global_param.yaml
  58. 4
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed.yaml
  59. 4
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_failed_with_retry.yaml
  60. 4
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_one_fake_task_success.yaml
  61. 36
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_three_parallel_three_fake_task_success.yaml
  62. 8
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_failed.yaml
  63. 8
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_parallel_fake_task_success.yaml
  64. 8
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_failed.yaml
  65. 8
      dolphinscheduler-master/src/test/resources/it/start/workflow_with_two_serial_fake_task_success.yaml
  66. 4
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_failed.yaml
  67. 4
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_one_fake_task_success.yaml
  68. 36
      dolphinscheduler-master/src/test/resources/it/stop/workflow_with_three_parallel_three_fake_task_success.yaml
  69. 35
      dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java
  70. 8
      dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java
  71. 2
      dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryAutoConfiguration.java
  72. 2
      dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java
  73. 1
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
  74. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java
  75. 11
      dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPoolTest.java

1
dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertServer.java

@ -56,6 +56,7 @@ public class AlertServer {
@PostConstruct
public void run() {
ServerLifeCycleManager.toRunning();
log.info("AlertServer is staring ...");
alertBootstrapService.start();
log.info("AlertServer is started ...");

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java

@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.api;
import org.apache.dolphinscheduler.api.metrics.ApiServerMetrics;
import org.apache.dolphinscheduler.common.CommonConfiguration;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.DefaultUncaughtExceptionHandler;
import org.apache.dolphinscheduler.dao.DaoConfiguration;
import org.apache.dolphinscheduler.dao.PluginDao;
@ -59,6 +60,7 @@ public class ApiApplicationServer {
@EventListener
public void run(ApplicationReadyEvent readyEvent) {
ServerLifeCycleManager.toRunning();
log.info("Received spring application context ready event will load taskPlugin and write to DB");
DataSourceProcessorProvider.initialize();
TaskPluginManager.loadTaskPlugin();

5
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/lifecycle/ServerLifeCycleManager.java

@ -32,6 +32,10 @@ public class ServerLifeCycleManager {
return serverStartupTime;
}
public static void toRunning() {
serverStatus = ServerStatus.RUNNING;
}
public static boolean isRunning() {
return serverStatus == ServerStatus.RUNNING;
}
@ -81,6 +85,7 @@ public class ServerLifeCycleManager {
if (serverStatus == ServerStatus.STOPPED) {
return false;
}
log.info("The current server status changed from {} to {}", serverStatus, ServerStatus.STOPPED);
serverStatus = ServerStatus.STOPPED;
return true;
}

6
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/BaseHeartBeatTask.java

@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.common.model;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import lombok.extern.slf4j.Slf4j;
@ -54,11 +53,6 @@ public abstract class BaseHeartBeatTask<T extends HeartBeat> extends BaseDaemonT
public void run() {
while (runningFlag) {
try {
if (!ServerLifeCycleManager.isRunning()) {
log.info("The current server status is {}, will not write heartBeatInfo into registry",
ServerLifeCycleManager.getServerStatus());
continue;
}
T heartBeat = getHeartBeat();
// if first time or heartBeat status changed, write heartBeatInfo into registry
if (System.currentTimeMillis() - lastWriteTime >= heartBeatInterval

11
dolphinscheduler-master/pom.xml

@ -312,6 +312,17 @@
</dependencies>
<build>
<testResources>
<testResource>
<directory>${project.basedir}/../dolphinscheduler-dao/src/main/resources</directory>
<includes>
<include>sql/**</include>
</includes>
</testResource>
<testResource>
<directory>${project.basedir}/src/test/resources</directory>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>

13
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java

@ -46,6 +46,7 @@ import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.util.Date;
import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
import lombok.extern.slf4j.Slf4j;
@ -106,6 +107,7 @@ public class MasterServer implements IStoppable {
*/
@PostConstruct
public void initialized() {
ServerLifeCycleManager.toRunning();
final long startupTime = System.currentTimeMillis();
// init rpc server
@ -150,11 +152,11 @@ public class MasterServer implements IStoppable {
log.info("MasterServer initialized successfully in {} ms", System.currentTimeMillis() - startupTime);
}
/**
* gracefully close
*
* @param cause close cause
*/
@PreDestroy
public void shutdown() {
close("MasterServer shutdown");
}
public void close(String cause) {
// set stop signal is true
// execute only once
@ -165,6 +167,7 @@ public class MasterServer implements IStoppable {
// thread sleep 3 seconds for thread quietly stop
ThreadUtils.sleep(Constants.SERVER_CLOSE_WAIT_TIME.toMillis());
try (
SystemEventBusFireWorker systemEventBusFireWorker1 = systemEventBusFireWorker;
WorkflowEngine workflowEngine1 = workflowEngine;
SchedulerApi closedSchedulerApi = schedulerApi;
MasterRpcServer closedRpcServer = masterRPCServer;

16
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java

@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.TaskGroup;
@ -80,7 +79,7 @@ import org.springframework.stereotype.Component;
*/
@Slf4j
@Component
public class TaskGroupCoordinator extends BaseDaemonThread {
public class TaskGroupCoordinator extends BaseDaemonThread implements AutoCloseable {
@Autowired
private RegistryClient registryClient;
@ -97,6 +96,8 @@ public class TaskGroupCoordinator extends BaseDaemonThread {
@Autowired
private WorkflowInstanceDao workflowInstanceDao;
private boolean flag = true;
private static int DEFAULT_LIMIT = 1000;
public TaskGroupCoordinator() {
@ -106,17 +107,15 @@ public class TaskGroupCoordinator extends BaseDaemonThread {
@Override
public synchronized void start() {
log.info("TaskGroupCoordinator starting...");
flag = true;
super.start();
log.info("TaskGroupCoordinator started...");
}
@Override
public void run() {
while (!ServerLifeCycleManager.isStopped()) {
while (flag) {
try {
if (!ServerLifeCycleManager.isRunning()) {
continue;
}
registryClient.getLock(RegistryNodeType.MASTER_TASK_GROUP_COORDINATOR_LOCK.getRegistryPath());
try {
StopWatch taskGroupCoordinatorRoundCost = StopWatch.createStarted();
@ -488,4 +487,9 @@ public class TaskGroupCoordinator extends BaseDaemonThread {
log.info("Success release TaskGroupQueue: {}", taskGroupQueue);
}
@Override
public void close() throws Exception {
flag = false;
log.info("TaskGroupCoordinator closed");
}
}

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java

@ -59,7 +59,8 @@ public class WorkflowEngine implements AutoCloseable {
try (
final CommandEngine commandEngine1 = commandEngine;
final WorkflowEventBusCoordinator workflowEventBusCoordinator1 = workflowEventBusCoordinator;
final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap;) {
final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap;
final TaskGroupCoordinator taskGroupCoordinator1 = taskGroupCoordinator) {
// closed the resource
}
}

13
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/CommandEngine.java

@ -21,7 +21,6 @@ import static java.util.concurrent.CompletableFuture.supplyAsync;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
@ -83,6 +82,8 @@ public class CommandEngine extends BaseDaemonThread implements AutoCloseable {
private ExecutorService commandHandleThreadPool;
private boolean flag = false;
protected CommandEngine() {
super("MasterCommandLoopThread");
}
@ -92,6 +93,7 @@ public class CommandEngine extends BaseDaemonThread implements AutoCloseable {
log.info("MasterSchedulerBootstrap starting..");
this.commandHandleThreadPool = ThreadUtils.newDaemonFixedThreadExecutor("MasterCommandHandleThreadPool",
Runtime.getRuntime().availableProcessors());
flag = true;
super.start();
log.info("MasterSchedulerBootstrap started...");
}
@ -99,20 +101,15 @@ public class CommandEngine extends BaseDaemonThread implements AutoCloseable {
@Override
public void close() throws Exception {
log.info("MasterSchedulerBootstrap stopping...");
flag = false;
log.info("MasterSchedulerBootstrap stopped...");
}
@Override
public void run() {
MasterServerLoadProtection serverLoadProtection = masterConfig.getServerLoadProtection();
while (!ServerLifeCycleManager.isStopped()) {
while (flag) {
try {
if (!ServerLifeCycleManager.isRunning()) {
// the current server is not at running status, cannot consume command.
log.warn("The current server is not at running status, cannot consumes commands.");
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
}
// todo: if the workflow event queue is much, we need to handle the back pressure
SystemMetrics systemMetrics = metricsProvider.getSystemMetrics();
if (serverLoadProtection.isOverload(systemMetrics)) {

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/AbstractCommandHandler.java

@ -91,8 +91,7 @@ public abstract class AbstractCommandHandler implements ICommandHandler {
protected void assembleWorkflowInstanceLifecycleListeners(
final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) {
workflowExecuteContextBuilder.setWorkflowInstanceLifecycleListeners(
workflowExecuteContextBuilder.getWorkflowInstanceLifecycleListeners());
workflowExecuteContextBuilder.setWorkflowInstanceLifecycleListeners(workflowLifecycleListeners);
}
protected void assembleWorkflowDefinition(

13
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/SystemEventBusFireWorker.java

@ -38,7 +38,7 @@ import org.springframework.stereotype.Component;
@Slf4j
@Component
@SuppressWarnings({"unchecked", "rawtypes"})
public class SystemEventBusFireWorker extends BaseDaemonThread {
public class SystemEventBusFireWorker extends BaseDaemonThread implements AutoCloseable {
@Autowired
private SystemEventBus systemEventBus;
@ -49,19 +49,22 @@ public class SystemEventBusFireWorker extends BaseDaemonThread {
@Autowired
private List<ISystemEventHandler> systemEventHandlers;
private static boolean flag = false;
public SystemEventBusFireWorker() {
super("SystemEventBusFireWorker");
}
@Override
public void start() {
flag = true;
super.start();
log.info("SystemEventBusFireWorker started");
}
@Override
public void run() {
while (!ServerLifeCycleManager.isStopped()) {
while (flag) {
final AbstractSystemEvent systemEvent;
try {
systemEvent = systemEventBus.take();
@ -99,4 +102,10 @@ public class SystemEventBusFireWorker extends BaseDaemonThread {
stopWatch.stop();
log.info("Fire SystemEvent: {} cost: {} ms", systemEvent, stopWatch.getTime());
}
@Override
public void close() {
flag = false;
log.info("SystemEventBusFireWorker closed");
}
}

1
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java

@ -126,6 +126,7 @@ public class TaskExecutionRunnable implements ITaskExecutionRunnable {
.withTaskInstance(taskInstance)
.build();
initializeTaskExecutionContext();
getWorkflowEventBus().publish(TaskStartLifecycleEvent.of(this));
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/listener/WorkflowSuccessLifecycleListener.java

@ -63,7 +63,7 @@ public class WorkflowSuccessLifecycleListener implements IWorkflowLifecycleListe
final ICommandParam commandParam =
JSONUtils.parseObject(workflowInstance.getCommandParam(), ICommandParam.class);
if (commandParam == null) {
log.warn("Command param: {} is invalid for workflow{}", workflowInstance.getCommandParam(),
log.warn("Command param: {} is invalid for workflow: {}", workflowInstance.getCommandParam(),
workflowInstance.getName());
return;
}

8
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java

@ -76,7 +76,13 @@ public class MasterRegistryClient implements AutoCloseable {
@Override
public void close() {
// TODO unsubscribe MasterRegistryDataListener
deregister();
if (masterHeartBeatTask != null) {
masterHeartBeatTask.shutdown();
}
if (registryClient.isConnected()) {
deregister();
}
log.info("Closed MasterRegistryClient");
}
/**

17
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/AbstractMasterIntegrationTest.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/AbstractMasterIntegrationTestCase.java

@ -19,12 +19,9 @@ package org.apache.dolphinscheduler.server.master;
import org.apache.dolphinscheduler.dao.DaoConfiguration;
import org.apache.curator.test.TestingServer;
import javax.annotation.PostConstruct;
import lombok.extern.slf4j.Slf4j;
import org.springframework.boot.test.context.SpringBootTest;
import org.springframework.context.annotation.Configuration;
import org.springframework.test.annotation.DirtiesContext;
/**
@ -32,19 +29,11 @@ import org.springframework.test.annotation.DirtiesContext;
* <p> Used to create a text environment to test master server.
* <p> In order to separate the environment for each text case, the context will be dirtied before each test method.
*/
@Slf4j
@SpringBootTest(classes = {
MasterServer.class,
DaoConfiguration.class})
@DirtiesContext(classMode = DirtiesContext.ClassMode.AFTER_EACH_TEST_METHOD)
public abstract class AbstractMasterIntegrationTest {
@Configuration
public static class RegistryServer {
public abstract class AbstractMasterIntegrationTestCase {
@PostConstruct
public void startEmbedRegistryServer() throws Exception {
final TestingServer server = new TestingServer(true);
System.setProperty("registry.zookeeper.connect-string", server.getConnectString());
}
}
}

2
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/Repository.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/Repository.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it;
package org.apache.dolphinscheduler.server.master.integration;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;

6
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowOperator.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it;
package org.apache.dolphinscheduler.server.master.integration;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Schedule;
@ -81,8 +81,8 @@ public class WorkflowOperator {
.startParamList(workflowBackfillDTO.getBackfillWorkflowCommandParam().getCommandParams())
.backfillTimeList(workflowBackfillDTO.getBackfillWorkflowCommandParam().getBackfillTimeList())
.build();
final WorkflowBackfillTriggerResponse backfillTriggerResponse =
workflowInstanceController.backfillTriggerWorkflow(backfillTriggerRequest);
final WorkflowBackfillTriggerResponse backfillTriggerResponse = workflowInstanceController
.backfillTriggerWorkflow(backfillTriggerRequest);
Assertions.assertThat(backfillTriggerResponse.isSuccess()).isTrue();
}

4
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContext.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowTestCaseContext.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it;
package org.apache.dolphinscheduler.server.master.integration;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
@ -33,7 +33,7 @@ import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowITContext {
public class WorkflowTestCaseContext {
private Project project;

26
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowITContextFactory.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/WorkflowTestCaseContextFactory.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it;
package org.apache.dolphinscheduler.server.master.integration;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
@ -45,7 +45,7 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class WorkflowITContextFactory {
public class WorkflowTestCaseContextFactory {
@Autowired
private ProjectDao projectDao;
@ -74,19 +74,19 @@ public class WorkflowITContextFactory {
@Autowired
private TaskInstanceDao taskInstanceDao;
public WorkflowITContext initializeContextFromYaml(final String yamlPath) {
final WorkflowITContext workflowITContext = YamlFactory.load(yamlPath);
initializeProjectToDB(workflowITContext.getProject());
initializeWorkflowDefinitionToDB(workflowITContext.getWorkflows());
initializeTaskDefinitionsToDB(workflowITContext.getTasks());
initializeTaskRelationsToDB(workflowITContext.getTaskRelations());
if (workflowITContext.getWorkflowInstance() != null) {
initializeWorkflowInstanceToDB(workflowITContext.getWorkflowInstance());
public WorkflowTestCaseContext initializeContextFromYaml(final String yamlPath) {
final WorkflowTestCaseContext workflowTestCaseContext = YamlFactory.load(yamlPath);
initializeProjectToDB(workflowTestCaseContext.getProject());
initializeWorkflowDefinitionToDB(workflowTestCaseContext.getWorkflows());
initializeTaskDefinitionsToDB(workflowTestCaseContext.getTasks());
initializeTaskRelationsToDB(workflowTestCaseContext.getTaskRelations());
if (workflowTestCaseContext.getWorkflowInstance() != null) {
initializeWorkflowInstanceToDB(workflowTestCaseContext.getWorkflowInstance());
}
if (CollectionUtils.isNotEmpty(workflowITContext.getTaskInstances())) {
initializeTaskInstancesToDB(workflowITContext.getTaskInstances());
if (CollectionUtils.isNotEmpty(workflowTestCaseContext.getTaskInstances())) {
initializeTaskInstancesToDB(workflowTestCaseContext.getTaskInstances());
}
return workflowITContext;
return workflowTestCaseContext;
}
private void initializeTaskInstancesToDB(List<TaskInstance> taskInstances) {

6
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/YamlFactory.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/YamlFactory.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it;
package org.apache.dolphinscheduler.server.master.integration;
import java.io.InputStream;
@ -26,13 +26,13 @@ import org.yaml.snakeyaml.Yaml;
public class YamlFactory {
@SneakyThrows
public static WorkflowITContext load(final String yamlRelativePath) {
public static WorkflowTestCaseContext load(final String yamlRelativePath) {
final Yaml yaml = new Yaml();
try (InputStream fis = YamlFactory.class.getResourceAsStream(yamlRelativePath)) {
if (fis == null) {
throw new IllegalArgumentException("Cannot find the file: " + yamlRelativePath + " under classpath");
}
return yaml.loadAs(fis, WorkflowITContext.class);
return yaml.loadAs(fis, WorkflowTestCaseContext.class);
}
}

22
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowBackfillIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowBackfillTestCase.java

@ -15,19 +15,18 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper;
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
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 org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
import org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.time.DateUtils;
@ -43,10 +42,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for scheduling a workflow from workflow definition.
*/
public class WorkflowBackfillIT extends AbstractMasterIntegrationTest {
public class WorkflowBackfillTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -54,14 +53,11 @@ public class WorkflowBackfillIT extends AbstractMasterIntegrationTest {
@Autowired
private Repository repository;
@Autowired
private ScheduleMapper scheduleMapper;
@Test
@DisplayName("Test backfill a workflow in asc order success")
public void testSerialBackfillWorkflow_with_oneSuccessTask() {
final String yaml = "/it/backfill/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder()

58
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceFailoverIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceFailoverTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static org.awaitility.Awaitility.await;
@ -24,12 +24,12 @@ 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.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
import org.apache.dolphinscheduler.server.master.engine.system.SystemEventBus;
import org.apache.dolphinscheduler.server.master.engine.system.event.GlobalMasterFailoverEvent;
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.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.StringUtils;
@ -41,10 +41,10 @@ import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceFailoverTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private SystemEventBus systemEventBus;
@ -55,7 +55,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -91,7 +91,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -131,7 +131,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -171,7 +171,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -205,7 +205,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -240,7 +240,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -275,11 +275,14 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
// Since the task take over failed
// So will create a new task instance and trigger it, but the workflow instance is ready pause
// The task will be paused.
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
@ -289,7 +292,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
.anySatisfy(workflowInstance -> {
Assertions
.assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.SUCCESS);
.isEqualTo(WorkflowExecutionStatus.PAUSE);
Assertions
.assertThat(workflowInstance.getName())
.isEqualTo("workflow_with_one_fake_task_success-20240816071251690");
@ -306,7 +309,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
Assertions
.assertThat(taskInstances.get(1))
.matches(t -> t.getState() == TaskExecutionStatus.SUCCESS)
.matches(t -> t.getState() == TaskExecutionStatus.PAUSE)
.matches(t -> t.getFlag() == Flag.YES);
});
@ -315,7 +318,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -350,7 +353,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -385,7 +388,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -420,7 +423,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -455,11 +458,14 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
// Since the task take over failed
// So will create a new task instance and trigger it, but the workflow instance is ready stop
// The task will be killed.
await()
.atMost(Duration.ofMinutes(1))
.untilAsserted(() -> {
@ -469,7 +475,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
.anySatisfy(workflowInstance -> {
Assertions
.assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.SUCCESS);
.isEqualTo(WorkflowExecutionStatus.STOP);
Assertions
.assertThat(workflowInstance.getName())
.isEqualTo("workflow_with_one_fake_task_success-20240816071251690");
@ -486,7 +492,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
Assertions
.assertThat(taskInstances.get(1))
.matches(t -> t.getState() == TaskExecutionStatus.SUCCESS)
.matches(t -> t.getState() == TaskExecutionStatus.KILL)
.matches(t -> t.getFlag() == Flag.YES);
});
@ -495,7 +501,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -530,7 +536,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));
@ -565,7 +571,7 @@ public class WorkflowInstanceFailoverIT extends AbstractMasterIntegrationTest {
@Test
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
systemEventBus.publish(GlobalMasterFailoverEvent.of(new Date()));

24
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstancePauseTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -26,12 +26,12 @@ 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;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import java.time.Duration;
import java.util.List;
@ -44,10 +44,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for pausing a workflow instance.
*/
public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
public class WorkflowInstancePauseTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -62,7 +62,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
@DisplayName("Test pause a workflow with one success task")
public void testPauseWorkflow_with_oneSuccessTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -113,7 +113,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
@DisplayName("Test pause a workflow with one failed task")
public void testPauseWorkflow_with_oneFailedTask() {
final String yaml = "/it/pause/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -164,7 +164,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
@DisplayName("Test pause a workflow with three parallel three fake tasks(A1->A2->A3, B1->B2->B3, C1->C2->C3) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -243,7 +243,7 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()

18
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverFailureTaskIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverFailureTaskTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -25,12 +25,12 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.StringUtils;
@ -45,10 +45,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for recover from failure tasks.
*/
public class WorkflowInstanceRecoverFailureTaskIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceRecoverFailureTaskTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -63,7 +63,7 @@ public class WorkflowInstanceRecoverFailureTaskIT extends AbstractMasterIntegrat
@DisplayName("Test recover from failure tasks")
public void testRepeatRunningWorkflow_with_taskOnly() {
final String yaml = "/it/recover_failure_tasks/failure_workflow_with_two_serial_fake_task.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final Integer workflowInstanceId = context.getWorkflowInstance().getId();
workflowOperator.recoverFailureTasks(workflowInstanceId);

18
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverPauseIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverPauseTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -26,12 +26,12 @@ 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.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import java.time.Duration;
import java.util.List;
@ -44,10 +44,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for pausing a workflow instance.
*/
public class WorkflowInstanceRecoverPauseIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceRecoverPauseTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -62,7 +62,7 @@ public class WorkflowInstanceRecoverPauseIT extends AbstractMasterIntegrationTes
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()

18
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRecoverStopIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRecoverStopTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -26,12 +26,12 @@ 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.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import java.time.Duration;
import java.util.List;
@ -44,10 +44,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for pausing a workflow instance.
*/
public class WorkflowInstanceRecoverStopIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceRecoverStopTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -62,7 +62,7 @@ public class WorkflowInstanceRecoverStopIT extends AbstractMasterIntegrationTest
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()

22
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceRepeatRunningIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceRepeatRunningTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -25,12 +25,12 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.StringUtils;
@ -45,10 +45,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for repeat running a workflow instance.
*/
public class WorkflowInstanceRepeatRunningIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceRepeatRunningTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -63,7 +63,7 @@ public class WorkflowInstanceRepeatRunningIT extends AbstractMasterIntegrationTe
@DisplayName("Test repeat running a workflow instance with one success task")
public void testRepeatRunningWorkflow_with_oneSuccessTask() {
final String yaml = "/it/repeat_running/success_workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final Integer workflowInstanceId = context.getWorkflowInstance().getId();
workflowOperator.repeatRunningWorkflowInstance(workflowInstanceId);
@ -109,7 +109,7 @@ public class WorkflowInstanceRepeatRunningIT extends AbstractMasterIntegrationTe
@DisplayName("Test repeat running a workflow instance with one failed task")
public void testRepeatRunningWorkflow_with_oneFailedTask() {
final String yaml = "/it/repeat_running/failed_workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final Integer workflowInstanceId = context.getWorkflowInstance().getId();
workflowOperator.repeatRunningWorkflowInstance(workflowInstanceId);
@ -156,7 +156,7 @@ public class WorkflowInstanceRepeatRunningIT extends AbstractMasterIntegrationTe
@DisplayName("Test repeat running a workflow instance with task only")
public void testRepeatRunningWorkflow_with_taskOnly() {
final String yaml = "/it/repeat_running/success_workflow_with_task_only.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final Integer workflowInstanceId = context.getWorkflowInstance().getId();
workflowOperator.repeatRunningWorkflowInstance(workflowInstanceId);

24
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowInstanceStopTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -27,12 +27,12 @@ import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import java.time.Duration;
@ -44,10 +44,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for stopping a workflow instance.
*/
public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
public class WorkflowInstanceStopTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -62,7 +62,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
@DisplayName("Test stop a workflow with one success task")
public void testStopWorkflow_with_oneSuccessTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -111,7 +111,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
@DisplayName("Test stop a workflow with one failed task")
public void testStopWorkflow_with_oneFailedTask() {
final String yaml = "/it/stop/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -160,7 +160,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
@DisplayName("Test stop a workflow with three parallel three fake tasks(A1->A2->A3, B1->B2->B3, C1->C2->C3) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -214,7 +214,7 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()

18
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowSchedulingIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowSchedulingTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static org.awaitility.Awaitility.await;
@ -28,11 +28,11 @@ import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
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 org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
import org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.time.DateUtils;
@ -49,10 +49,10 @@ import org.springframework.beans.factory.annotation.Autowired;
/**
* The integration test for scheduling a workflow from workflow definition.
*/
public class WorkflowSchedulingIT extends AbstractMasterIntegrationTest {
public class WorkflowSchedulingTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -67,7 +67,7 @@ public class WorkflowSchedulingIT extends AbstractMasterIntegrationTest {
@DisplayName("Test scheduling a workflow with one fake task(A) success")
public void testSchedulingWorkflow_with_oneSuccessTask() {
final String yaml = "/it/scheduling/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final Schedule schedule = Schedule.builder()

42
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java → dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/integration/cases/WorkflowStartTestCase.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.it.cases;
package org.apache.dolphinscheduler.server.master.integration.cases;
import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await;
@ -30,12 +30,12 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.DataType;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTestCase;
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 org.apache.dolphinscheduler.server.master.integration.Repository;
import org.apache.dolphinscheduler.server.master.integration.WorkflowOperator;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContext;
import org.apache.dolphinscheduler.server.master.integration.WorkflowTestCaseContextFactory;
import org.apache.commons.lang3.time.DateUtils;
@ -54,10 +54,10 @@ import com.google.common.collect.Lists;
* <p> In each test method, will create different workflow from yaml, and then trigger it, and do assertions.
* <p> The method name should be clear to describe the test scenario.
*/
public class WorkflowStartIT extends AbstractMasterIntegrationTest {
public class WorkflowStartTestCase extends AbstractMasterIntegrationTestCase {
@Autowired
private WorkflowITContextFactory workflowITContextFactory;
private WorkflowTestCaseContextFactory workflowTestCaseContextFactory;
@Autowired
private WorkflowOperator workflowOperator;
@ -72,7 +72,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with one fake task(A) success")
public void testStartWorkflow_with_oneSuccessTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -103,7 +103,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition parentWorkflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -155,7 +155,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition parentWorkflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -207,7 +207,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -243,7 +243,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow which using command params")
public void testStartWorkflow_usingCommandParam() {
final String yaml = "/it/start/workflow_with_global_param.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
@ -288,7 +288,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with one fake task(A) failed")
public void testStartWorkflow_with_oneFailedTask() {
final String yaml = "/it/start/workflow_with_one_fake_task_failed.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -319,7 +319,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with one fake task(A) failed")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -372,7 +372,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with two serial fake tasks(A -> B) success")
public void testStartWorkflow_with_twoSerialSuccessTask() {
String yaml = "/it/start/workflow_with_two_serial_fake_task_success.yaml";
final WorkflowITContext context = workflowITContextFactory.initializeContextFromYaml(yaml);
final WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -411,7 +411,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with two serial fake tasks(A(failed) -> B) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -442,7 +442,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with two parallel fake tasks(A, B) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -480,7 +480,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with two parallel fake tasks(A(failed), B(failed)) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -519,7 +519,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with three parallel three fake tasks(A1->A2->A3, B1->B2->B3, C1->C2->C3) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final WorkflowOperator.WorkflowTriggerDTO workflowTriggerDTO = WorkflowOperator.WorkflowTriggerDTO.builder()
@ -586,7 +586,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
@DisplayName("Test start a workflow with three parallel three fake tasks(A1->A2->A3, B1->B2->B3, C1->C2->C3) success")
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 WorkflowTestCaseContext context = workflowTestCaseContextFactory.initializeContextFromYaml(yaml);
final WorkflowDefinition workflow = context.getWorkflows().get(0);
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()

6
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java

@ -79,10 +79,9 @@ class GlobalTaskDispatchWaitingQueueLooperTest {
doNothing().when(taskDispatcher).dispatchTask(any());
when(globalTaskDispatchWaitingQueue.takeTaskExecuteRunnable()).thenReturn(defaultTaskExecuteRunnable);
globalTaskDispatchWaitingQueueLooper.start();
globalTaskDispatchWaitingQueueLooper.doDispatch();
await().during(ofSeconds(1))
.untilAsserted(() -> verify(taskDispatchFactory, never()).getTaskDispatcher(taskInstance));
globalTaskDispatchWaitingQueueLooper.close();
}
@Test
@ -99,12 +98,11 @@ class GlobalTaskDispatchWaitingQueueLooperTest {
doNothing().when(taskDispatcher).dispatchTask(any());
when(globalTaskDispatchWaitingQueue.takeTaskExecuteRunnable()).thenReturn(defaultTaskExecuteRunnable);
globalTaskDispatchWaitingQueueLooper.start();
globalTaskDispatchWaitingQueueLooper.doDispatch();
await().atMost(ofSeconds(1)).untilAsserted(() -> {
verify(taskDispatchFactory, atLeastOnce()).getTaskDispatcher(any(TaskInstance.class));
verify(taskDispatcher, atLeastOnce()).dispatchTask(any(ITaskExecutionRunnable.class));
});
globalTaskDispatchWaitingQueueLooper.close();
}

13
dolphinscheduler-master/src/test/resources/application.yaml

@ -42,18 +42,7 @@ mybatis-plus:
banner: false
registry:
type: zookeeper
zookeeper:
namespace: dolphinscheduler
connect-string: localhost:2181
retry-policy:
base-sleep-time: 60ms
max-sleep: 300ms
max-retries: 5
session-timeout: 30s
connection-timeout: 9s
block-until-connected: 600ms
digest: ~
type: jdbc
master:
listen-port: 5678

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_PAUSE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_PAUSE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_PAUSE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_PAUSE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_PAUSE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_STOP
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_STOP
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_STOP
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_STOP
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: READY_STOP
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -99,8 +99,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: RUNNING_EXECUTION
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: RUNNING_EXECUTION
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -101,8 +101,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: RUNNING_EXECUTION
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -100,8 +100,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: RUNNING_EXECUTION
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -98,8 +98,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: RUNNING_EXECUTION
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -101,8 +101,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -139,8 +139,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -148,8 +148,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 4
@ -157,8 +157,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 4
preTaskVersion: 1
postTaskCode: 7
@ -166,8 +166,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2
@ -175,8 +175,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 2
preTaskVersion: 1
postTaskCode: 5
@ -184,8 +184,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 5
preTaskVersion: 1
postTaskCode: 8
@ -193,8 +193,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
@ -202,8 +202,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 3
preTaskVersion: 1
postTaskCode: 6
@ -211,8 +211,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 6
preTaskVersion: 1
postTaskCode: 9

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

@ -26,8 +26,8 @@ project:
workflowInstance:
id: 1
name: workflow_with_two_serial_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: FAILURE
recovery: NO
@ -39,7 +39,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -51,8 +51,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_two_parallel_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_two_parallel_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -111,8 +111,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -120,8 +120,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: FAILURE
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -101,8 +101,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -39,8 +39,8 @@ workflows:
workflowInstance:
id: 1
name: workflow_with_one_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: SUCCESS
recovery: NO
@ -52,7 +52,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_POST
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -64,8 +64,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_one_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_one_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -101,8 +101,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -27,8 +27,8 @@ project:
workflowInstance:
id: 1
name: workflow_with_two_parallel_fake_task_success-20240816071251690
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
projectCode: 1
state: SUCCESS
recovery: NO
@ -40,7 +40,7 @@ workflowInstance:
commandParam: '{"commandType":"START_PROCESS","startNodes":[1],"commandParams":[],"timeZone":"UTC"}'
taskDependType: TASK_ONLY
commandStartTime: 2024-08-16 07:12:52
isSubProcess: NO
isSubWorkflow: NO
executorId: 1
historyCmd: START_PROCESS
workerGroup: default
@ -52,8 +52,8 @@ taskInstances:
- id: 1
name: A
taskType: LogicFakeTask
processInstanceId: 1
processInstanceName: workflow_with_two_parallel_fake_task_success-20240816071251690
workflowInstanceId: 1
workflowInstanceName: workflow_with_two_parallel_fake_task_success-20240816071251690
projectCode: 1
taskCode: 1
taskDefinitionVersion: 1
@ -112,8 +112,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -121,8 +121,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -65,8 +65,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -74,8 +74,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -53,8 +53,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -139,8 +139,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -148,8 +148,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 4
@ -157,8 +157,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 4
preTaskVersion: 1
postTaskCode: 7
@ -166,8 +166,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2
@ -175,8 +175,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 2
preTaskVersion: 1
postTaskCode: 5
@ -184,8 +184,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 5
preTaskVersion: 1
postTaskCode: 8
@ -193,8 +193,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
@ -202,8 +202,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 3
preTaskVersion: 1
postTaskCode: 6
@ -211,8 +211,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 6
preTaskVersion: 1
postTaskCode: 9

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

@ -62,8 +62,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -71,8 +71,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2

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

@ -62,8 +62,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -71,8 +71,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2

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

@ -62,8 +62,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -71,8 +71,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2

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

@ -61,8 +61,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -70,8 +70,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 2

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -51,8 +51,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1

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

@ -139,8 +139,8 @@ tasks:
taskRelations:
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 1
@ -148,8 +148,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 1
preTaskVersion: 1
postTaskCode: 4
@ -157,8 +157,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 4
preTaskVersion: 1
postTaskCode: 7
@ -166,8 +166,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 2
@ -175,8 +175,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 2
preTaskVersion: 1
postTaskCode: 5
@ -184,8 +184,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 5
preTaskVersion: 1
postTaskCode: 8
@ -193,8 +193,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 0
preTaskVersion: 0
postTaskCode: 3
@ -202,8 +202,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 3
preTaskVersion: 1
postTaskCode: 6
@ -211,8 +211,8 @@ taskRelations:
createTime: 2024-08-12 00:00:00
updateTime: 2024-08-12 00:00:00
- projectCode: 1
processDefinitionCode: 1
processDefinitionVersion: 1
workflowDefinitionCode: 1
workflowDefinitionVersion: 1
preTaskCode: 6
preTaskVersion: 1
postTaskCode: 9

35
dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java

@ -17,8 +17,6 @@
package org.apache.dolphinscheduler.registry.api;
import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.common.IStoppable;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.model.AlertServerHeartBeat;
@ -49,8 +47,6 @@ import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
import com.google.common.base.Strings;
@Component
@Slf4j
public class RegistryClient {
@ -164,26 +160,6 @@ public class RegistryClient {
.anyMatch(it -> it.contains(host));
}
public Collection<String> getMasterNodesDirectly() {
return getChildrenKeys(RegistryNodeType.MASTER.getRegistryPath());
}
/**
* get host ip:port, path format: parentPath/ip:port
*
* @param path path
* @return host ip:port, string format: parentPath/ip:port
*/
public String getHostByEventDataPath(String path) {
checkArgument(!Strings.isNullOrEmpty(path), "path cannot be null or empty");
final String[] pathArray = path.split(Constants.SINGLE_SLASH);
checkArgument(pathArray.length >= 1, "cannot parse path: %s", path);
return pathArray[pathArray.length - 1];
}
public void close() throws IOException {
registry.close();
}
@ -213,6 +189,9 @@ public class RegistryClient {
}
public boolean getLock(String key) {
if (!registry.isConnected()) {
throw new IllegalStateException("The registry is not connected");
}
return registry.acquireLock(key);
}
@ -228,14 +207,6 @@ public class RegistryClient {
return stoppable;
}
public boolean isMasterPath(String path) {
return path != null && path.startsWith(RegistryNodeType.MASTER.getRegistryPath() + Constants.SINGLE_SLASH);
}
public boolean isWorkerPath(String path) {
return path != null && path.startsWith(RegistryNodeType.WORKER.getRegistryPath() + Constants.SINGLE_SLASH);
}
public Collection<String> getChildrenKeys(final String key) {
return registry.children(key);
}

8
dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java

@ -26,6 +26,7 @@ import org.apache.dolphinscheduler.registry.api.RegistryException;
import org.apache.dolphinscheduler.registry.api.SubscribeListener;
import org.apache.commons.lang3.time.DurationUtils;
import org.apache.commons.lang3.time.StopWatch;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.framework.api.ACLProvider;
@ -101,13 +102,18 @@ final class ZookeeperRegistry implements Registry {
@Override
public void start() {
final StopWatch stopWatch = StopWatch.createStarted();
client.start();
try {
if (!client.blockUntilConnected(DurationUtils.toMillisInt(properties.getBlockUntilConnected()),
MILLISECONDS)) {
client.close();
throw new RegistryException("zookeeper connect failed in : " + properties.getConnectString() + "ms");
throw new RegistryException(
"zookeeper connect failed to: " + properties.getConnectString() + " in : "
+ properties.getBlockUntilConnected() + "ms");
}
stopWatch.stop();
log.info("ZookeeperRegistry started at: {}/ms", stopWatch.getTime());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RegistryException("Zookeeper registry start failed", e);

2
dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryAutoConfiguration.java

@ -39,7 +39,7 @@ public class ZookeeperRegistryAutoConfiguration {
@Bean
@ConditionalOnMissingBean(value = Registry.class)
public ZookeeperRegistry zookeeperRegistry(ZookeeperRegistryProperties zookeeperRegistryProperties) {
public Registry zookeeperRegistry(ZookeeperRegistryProperties zookeeperRegistryProperties) {
ZookeeperRegistry zookeeperRegistry = new ZookeeperRegistry(zookeeperRegistryProperties);
zookeeperRegistry.start();
return zookeeperRegistry;

2
dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java

@ -39,7 +39,7 @@ import org.springframework.validation.Validator;
@Configuration
@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "zookeeper")
@ConfigurationProperties(prefix = "registry")
class ZookeeperRegistryProperties implements Validator {
public class ZookeeperRegistryProperties implements Validator {
private ZookeeperProperties zookeeper = new ZookeeperProperties();

1
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java

@ -85,6 +85,7 @@ public class WorkerServer implements IStoppable {
@PostConstruct
public void run() {
ServerLifeCycleManager.toRunning();
this.workerRpcServer.start();
TaskPluginManager.loadTaskPlugin();
DataSourceProcessorProvider.initialize();

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java

@ -134,7 +134,7 @@ public class WorkerRegistryClient implements AutoCloseable {
workerHeartBeatTask.shutdown();
}
registryClient.close();
log.info("Worker registry client closed");
log.info("Closed WorkerRegistryClient");
}
public boolean isAvailable() {

11
dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPoolTest.java

@ -30,6 +30,7 @@ import org.apache.commons.lang3.RandomUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import com.google.common.truth.Truth;
@ -46,7 +47,7 @@ class WorkerTaskExecutorThreadPoolTest {
final int totalTaskCount = RandomUtils.nextInt(1, 10000);
final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
new WorkerTaskExecutorThreadPool(workerConfig, new WorkerMessageSender());
new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
// submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is CONTINUE
// after submit execThreadCount task, the thread pool is overload
for (int i = 1; i <= totalTaskCount; i++) {
@ -67,7 +68,7 @@ class WorkerTaskExecutorThreadPoolTest {
final int totalTaskCount = RandomUtils.nextInt(1, 10000);
final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
new WorkerTaskExecutorThreadPool(workerConfig, new WorkerMessageSender());
new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
// submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is CONTINUE
// all task will be submitted success
for (int i = 1; i <= totalTaskCount; i++) {
@ -83,7 +84,7 @@ class WorkerTaskExecutorThreadPoolTest {
final int totalTaskCount = RandomUtils.nextInt(1, 10000);
final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.REJECT);
final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
new WorkerTaskExecutorThreadPool(workerConfig, new WorkerMessageSender());
new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
// submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is REJECT
// only the front execThreadCount task will be submitted success
for (int i = 1; i <= totalTaskCount; i++) {
@ -104,7 +105,7 @@ class WorkerTaskExecutorThreadPoolTest {
final int totalTaskCount = RandomUtils.nextInt(1, 10000);
final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
new WorkerTaskExecutorThreadPool(workerConfig, new WorkerMessageSender());
new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
Truth.assertThat(workerTaskExecutorThreadPool.getWaitingTaskExecutorSize()).isEqualTo(0);
for (int i = 1; i <= totalTaskCount; i++) {
@ -126,7 +127,7 @@ class WorkerTaskExecutorThreadPoolTest {
final int totalTaskCount = RandomUtils.nextInt(1, 10000);
WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
new WorkerTaskExecutorThreadPool(workerConfig, new WorkerMessageSender());
new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
Truth.assertThat(workerTaskExecutorThreadPool.getRunningTaskExecutorSize()).isEqualTo(0);
for (int i = 1; i <= totalTaskCount; i++) {

Loading…
Cancel
Save