Browse Source

[DSIP-67] Use command to trigger workflow instance rather generate workflow instance (#16523)

dev
Wenjun Ruan 3 months ago committed by GitHub
parent
commit
20a0e50cce
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 4
      dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java
  2. 19
      dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ProcessInstanceAPITest.java
  3. 15
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java
  4. 108
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/BackfillWorkflowExecutorDelegate.java
  5. 12
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/PauseWorkflowInstanceExecutorDelegate.java
  6. 12
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/StopWorkflowInstanceExecutorDelegate.java
  7. 71
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/TriggerWorkflowExecutorDelegate.java
  8. 9
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
  9. 9
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/AlertPluginInstanceServiceImpl.java
  10. 17
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
  11. 21
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/LoggerServiceImpl.java
  12. 9
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/MetricsCleanUpServiceImpl.java
  13. 27
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
  14. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/validator/workflow/TriggerWorkflowDTO.java
  15. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/validator/workflow/TriggerWorkflowRequestTransformer.java
  16. 10
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowInstanceExecuteFunctionControllerTest.java
  17. 1
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java
  18. 2
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkflowExecutionStatus.java
  19. 35
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Command.java
  20. 63
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TriggerRelation.java
  21. 72
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapper.java
  22. 23
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/UserDao.java
  23. 36
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/UserDaoImpl.java
  24. 61
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapper.xml
  25. 16
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
  26. 16
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
  27. 15
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
  28. 2
      dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql
  29. 2
      dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql
  30. 134
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapperTest.java
  31. 22
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/Clients.java
  32. 17
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/test/java/org/apache/dolphinscheduler/extract/base/client/ClientsTest.java
  33. 53
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowControlClient.java
  34. 39
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowInstanceController.java
  35. 83
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowBackfillTriggerRequest.java
  36. 51
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowBackfillTriggerResponse.java
  37. 2
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstancePauseRequest.java
  38. 2
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstancePauseResponse.java
  39. 2
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstanceStopRequest.java
  40. 2
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstanceStopResponse.java
  41. 77
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowManualTriggerRequest.java
  42. 50
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowManualTriggerResponse.java
  43. 75
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowScheduleTriggerRequest.java
  44. 51
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowScheduleTriggerResponse.java
  45. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java
  46. 7
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/CommandEngine.java
  47. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/AbstractCommandHandler.java
  48. 21
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/BackfillWorkflowCommandHandler.java
  49. 5
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/ReRunWorkflowCommandHandler.java
  50. 53
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/RunWorkflowCommandHandler.java
  51. 17
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/ScheduleWorkflowCommandHandler.java
  52. 6
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java
  53. 6
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java
  54. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java
  55. 37
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/listener/WorkflowSuccessLifecycleListener.java
  56. 59
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/IWorkflowExecutionRunnable.java
  57. 50
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/WorkflowExecutionRunnable.java
  58. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/WorkflowExecutionRunnableFactory.java
  59. 1
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/IWorkflowStateAction.java
  60. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowStateActionFactory.java
  61. 121
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowSubmittedStateAction.java
  62. 86
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/AbstractWorkflowTrigger.java
  63. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/IWorkflowTrigger.java
  64. 114
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowBackfillTrigger.java
  65. 111
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowManualTrigger.java
  66. 106
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowScheduleTrigger.java
  67. 14
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutionEventListenerImpl.java
  68. 126
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/WorkflowControlClient.java
  69. 81
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/WorkflowInstanceControllerImpl.java
  70. 5
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContext.java
  71. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
  72. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteDispatchEventSender.java
  73. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java
  74. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFailedEventSender.java
  75. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionKilledEventSender.java
  76. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionPausedEventSender.java
  77. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionSuccessEventSender.java
  78. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
  79. 20
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
  80. 14
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/Repository.java
  81. 81
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/WorkflowOperator.java
  82. 91
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstancePauseIT.java
  83. 70
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowInstanceStopIT.java
  84. 22
      dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/cases/WorkflowStartIT.java
  85. 7
      dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/RpcBenchMarkTest.java
  86. 12
      dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java
  87. 5
      dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/pom.xml
  88. 49
      dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java
  89. 12
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
  90. 40
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/TriggerRelationService.java
  91. 94
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/TriggerRelationServiceImpl.java
  92. 3
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java
  93. 97
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/TriggerRelationServiceTest.java
  94. 5
      dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag-startup-param.tsx
  95. 4
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionDispatchEventSender.java
  96. 4
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionFailedEventSender.java
  97. 4
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionKilledEventSender.java
  98. 4
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionPausedEventSender.java
  99. 4
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionSuccessEventSender.java
  100. 12
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java
  101. Some files were not shown because too many files have changed in this diff Show More

4
dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java

@ -73,7 +73,7 @@ public class ExecutorAPITest {
private static long processDefinitionCode; private static long processDefinitionCode;
private static long triggerCode; private static List<Integer> workflowInstanceIds;
@BeforeAll @BeforeAll
public static void setup() { public static void setup() {
@ -138,7 +138,7 @@ public class ExecutorAPITest {
processDefinitionCode, scheduleTime, FailureStrategy.END, WarningType.NONE); processDefinitionCode, scheduleTime, FailureStrategy.END, WarningType.NONE);
Assertions.assertTrue(startProcessInstanceResponse.getBody().getSuccess()); Assertions.assertTrue(startProcessInstanceResponse.getBody().getSuccess());
triggerCode = (long) startProcessInstanceResponse.getBody().getData(); workflowInstanceIds = (List<Integer>) startProcessInstanceResponse.getBody().getData();
} catch (Exception e) { } catch (Exception e) {
log.error("failed", e); log.error("failed", e);
Assertions.fail(); Assertions.fail();

19
dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ProcessInstanceAPITest.java

@ -19,6 +19,7 @@
package org.apache.dolphinscheduler.api.test.cases; package org.apache.dolphinscheduler.api.test.cases;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import org.apache.dolphinscheduler.api.test.core.DolphinScheduler; import org.apache.dolphinscheduler.api.test.core.DolphinScheduler;
@ -44,6 +45,7 @@ import java.text.SimpleDateFormat;
import java.util.Date; import java.util.Date;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
@ -51,7 +53,6 @@ import lombok.extern.slf4j.Slf4j;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Order;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.testcontainers.shaded.org.awaitility.Awaitility; import org.testcontainers.shaded.org.awaitility.Awaitility;
@ -105,7 +106,7 @@ public class ProcessInstanceAPITest {
@Test @Test
@Order(1) @Order(1)
public void testQueryProcessInstancesByTriggerCode() { public void testQueryProcessInstancesByWorkflowInstanceId() {
try { try {
// create test project // create test project
HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test"); HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test");
@ -145,6 +146,10 @@ public class ProcessInstanceAPITest {
HttpResponse startProcessInstanceResponse = executorPage.startProcessInstance(loginUser, projectCode, HttpResponse startProcessInstanceResponse = executorPage.startProcessInstance(loginUser, projectCode,
processDefinitionCode, scheduleTime, FailureStrategy.END, WarningType.NONE); processDefinitionCode, scheduleTime, FailureStrategy.END, WarningType.NONE);
assertTrue(startProcessInstanceResponse.getBody().getSuccess()); assertTrue(startProcessInstanceResponse.getBody().getSuccess());
final List<Integer> workflowInstanceIds = (List<Integer>) startProcessInstanceResponse.getBody().getData();
assertEquals(1, workflowInstanceIds.size());
processInstanceId = workflowInstanceIds.get(0);
// make sure process instance has completed and successfully persisted into db // make sure process instance has completed and successfully persisted into db
Awaitility.await() Awaitility.await()
@ -152,10 +157,11 @@ public class ProcessInstanceAPITest {
.untilAsserted(() -> { .untilAsserted(() -> {
// query workflow instance by trigger code // query workflow instance by trigger code
HttpResponse queryProcessInstanceListResponse = HttpResponse queryProcessInstanceListResponse =
processInstancePage.queryProcessInstanceList(loginUser, projectCode, 1, 10); processInstancePage.queryProcessInstanceById(loginUser, projectCode, processInstanceId);
assertTrue(queryProcessInstanceListResponse.getBody().getSuccess()); assertTrue(queryProcessInstanceListResponse.getBody().getSuccess());
assertTrue(queryProcessInstanceListResponse.getBody().getData().toString() final Map<String, Object> workflowInstance =
.contains("test_import")); (Map<String, Object>) queryProcessInstanceListResponse.getBody().getData();
assertEquals("SUCCESS", workflowInstance.get("state"));
}); });
} catch (Exception e) { } catch (Exception e) {
log.error("failed", e); log.error("failed", e);
@ -174,7 +180,6 @@ public class ProcessInstanceAPITest {
@Test @Test
@Order(3) @Order(3)
@Disabled
public void testQueryTaskListByProcessId() { public void testQueryTaskListByProcessId() {
HttpResponse queryTaskListByProcessIdResponse = HttpResponse queryTaskListByProcessIdResponse =
processInstancePage.queryTaskListByProcessId(loginUser, projectCode, processInstanceId); processInstancePage.queryTaskListByProcessId(loginUser, projectCode, processInstanceId);
@ -184,7 +189,6 @@ public class ProcessInstanceAPITest {
@Test @Test
@Order(4) @Order(4)
@Disabled
public void testQueryProcessInstanceById() { public void testQueryProcessInstanceById() {
HttpResponse queryProcessInstanceByIdResponse = HttpResponse queryProcessInstanceByIdResponse =
processInstancePage.queryProcessInstanceById(loginUser, projectCode, processInstanceId); processInstancePage.queryProcessInstanceById(loginUser, projectCode, processInstanceId);
@ -194,7 +198,6 @@ public class ProcessInstanceAPITest {
@Test @Test
@Order(5) @Order(5)
@Disabled
public void testDeleteProcessInstanceById() { public void testDeleteProcessInstanceById() {
HttpResponse deleteProcessInstanceByIdResponse = HttpResponse deleteProcessInstanceByIdResponse =
processInstancePage.deleteProcessInstanceById(loginUser, projectCode, processInstanceId); processInstancePage.deleteProcessInstanceById(loginUser, projectCode, processInstanceId);

15
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java

@ -64,6 +64,8 @@ import org.springframework.web.bind.annotation.RequestParam;
import org.springframework.web.bind.annotation.ResponseStatus; import org.springframework.web.bind.annotation.ResponseStatus;
import org.springframework.web.bind.annotation.RestController; import org.springframework.web.bind.annotation.RestController;
import com.google.common.collect.Lists;
import io.swagger.v3.oas.annotations.Operation; import io.swagger.v3.oas.annotations.Operation;
import io.swagger.v3.oas.annotations.Parameter; import io.swagger.v3.oas.annotations.Parameter;
import io.swagger.v3.oas.annotations.Parameters; import io.swagger.v3.oas.annotations.Parameters;
@ -129,7 +131,7 @@ public class ExecutorController extends BaseController {
@ResponseStatus(HttpStatus.OK) @ResponseStatus(HttpStatus.OK)
@ApiException(START_PROCESS_INSTANCE_ERROR) @ApiException(START_PROCESS_INSTANCE_ERROR)
@OperatorLog(auditType = AuditType.PROCESS_START) @OperatorLog(auditType = AuditType.PROCESS_START)
public Result<Long> triggerWorkflowDefinition(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser, public Result<List<Integer>> triggerWorkflowDefinition(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam(value = "processDefinitionCode") long processDefinitionCode, @RequestParam(value = "processDefinitionCode") long processDefinitionCode,
@RequestParam(value = "scheduleTime") String scheduleTime, @RequestParam(value = "scheduleTime") String scheduleTime,
@RequestParam(value = "failureStrategy") FailureStrategy failureStrategy, @RequestParam(value = "failureStrategy") FailureStrategy failureStrategy,
@ -170,7 +172,8 @@ public class ExecutorController extends BaseController {
.dryRun(Flag.of(dryRun)) .dryRun(Flag.of(dryRun))
.testFlag(Flag.of(testFlag)) .testFlag(Flag.of(testFlag))
.build(); .build();
return Result.success(execService.triggerWorkflowDefinition(workflowTriggerRequest)); return Result
.success(Lists.newArrayList(execService.triggerWorkflowDefinition(workflowTriggerRequest)));
case COMPLEMENT_DATA: case COMPLEMENT_DATA:
final WorkflowBackFillRequest workflowBackFillRequest = WorkflowBackFillRequest.builder() final WorkflowBackFillRequest workflowBackFillRequest = WorkflowBackFillRequest.builder()
.loginUser(loginUser) .loginUser(loginUser)
@ -250,7 +253,7 @@ public class ExecutorController extends BaseController {
@ResponseStatus(HttpStatus.OK) @ResponseStatus(HttpStatus.OK)
@ApiException(BATCH_START_PROCESS_INSTANCE_ERROR) @ApiException(BATCH_START_PROCESS_INSTANCE_ERROR)
@OperatorLog(auditType = AuditType.PROCESS_BATCH_START) @OperatorLog(auditType = AuditType.PROCESS_BATCH_START)
public Result<List<Long>> batchTriggerWorkflowDefinitions(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser, public Result<List<Integer>> batchTriggerWorkflowDefinitions(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam(value = "processDefinitionCodes") String processDefinitionCodes, @RequestParam(value = "processDefinitionCodes") String processDefinitionCodes,
@RequestParam(value = "scheduleTime") String scheduleTime, @RequestParam(value = "scheduleTime") String scheduleTime,
@RequestParam(value = "failureStrategy") FailureStrategy failureStrategy, @RequestParam(value = "failureStrategy") FailureStrategy failureStrategy,
@ -276,9 +279,9 @@ public class ExecutorController extends BaseController {
.map(Long::parseLong) .map(Long::parseLong)
.collect(Collectors.toList()); .collect(Collectors.toList());
List<Long> result = new ArrayList<>(); List<Integer> result = new ArrayList<>();
for (Long workflowDefinitionCode : workflowDefinitionCodes) { for (Long workflowDefinitionCode : workflowDefinitionCodes) {
Result<Long> triggerCodeResult = triggerWorkflowDefinition(loginUser, Result<List<Integer>> workflowInstanceIds = triggerWorkflowDefinition(loginUser,
workflowDefinitionCode, workflowDefinitionCode,
scheduleTime, scheduleTime,
failureStrategy, failureStrategy,
@ -299,7 +302,7 @@ public class ExecutorController extends BaseController {
complementDependentMode, complementDependentMode,
allLevelDependent, allLevelDependent,
executionOrder); executionOrder);
result.add(triggerCodeResult.getData()); result.addAll(workflowInstanceIds.getData());
} }
return Result.success(result); return Result.success(result);
} }

108
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/BackfillWorkflowExecutorDelegate.java

@ -17,20 +17,25 @@
package org.apache.dolphinscheduler.api.executor.workflow; package org.apache.dolphinscheduler.api.executor.workflow;
import org.apache.dolphinscheduler.api.exceptions.ServiceException;
import org.apache.dolphinscheduler.api.validator.workflow.BackfillWorkflowDTO; import org.apache.dolphinscheduler.api.validator.workflow.BackfillWorkflowDTO;
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode; import org.apache.dolphinscheduler.common.enums.ComplementDependentMode;
import org.apache.dolphinscheduler.common.enums.ExecutionOrder; import org.apache.dolphinscheduler.common.enums.ExecutionOrder;
import org.apache.dolphinscheduler.common.enums.RunMode; import org.apache.dolphinscheduler.common.enums.RunMode;
import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import java.time.ZonedDateTime; import java.time.ZonedDateTime;
import java.util.Collections; import java.util.Collections;
import java.util.Date;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -43,7 +48,7 @@ import com.google.common.collect.Lists;
@Slf4j @Slf4j
@Component @Component
public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<BackfillWorkflowDTO, Void> { public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<BackfillWorkflowDTO, List<Integer>> {
@Autowired @Autowired
private CommandDao commandDao; private CommandDao commandDao;
@ -51,18 +56,20 @@ public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<Backf
@Autowired @Autowired
private ProcessService processService; private ProcessService processService;
@Autowired
private RegistryClient registryClient;
@Override @Override
public Void execute(final BackfillWorkflowDTO backfillWorkflowDTO) { public List<Integer> execute(final BackfillWorkflowDTO backfillWorkflowDTO) {
// todo: directly call the master api to do backfill // todo: directly call the master api to do backfill
if (backfillWorkflowDTO.getBackfillParams().getRunMode() == RunMode.RUN_MODE_SERIAL) { if (backfillWorkflowDTO.getBackfillParams().getRunMode() == RunMode.RUN_MODE_SERIAL) {
doSerialBackfillWorkflow(backfillWorkflowDTO); return doSerialBackfillWorkflow(backfillWorkflowDTO);
} else { } else {
doParallemBackfillWorkflow(backfillWorkflowDTO); return doParallemBackfillWorkflow(backfillWorkflowDTO);
} }
return null;
} }
private void doSerialBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) { private List<Integer> doSerialBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) {
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
final List<ZonedDateTime> backfillTimeList = backfillParams.getBackfillDateList(); final List<ZonedDateTime> backfillTimeList = backfillParams.getBackfillDateList();
if (backfillParams.getExecutionOrder() == ExecutionOrder.DESC_ORDER) { if (backfillParams.getExecutionOrder() == ExecutionOrder.DESC_ORDER) {
@ -71,17 +78,13 @@ public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<Backf
Collections.sort(backfillTimeList); Collections.sort(backfillTimeList);
} }
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder() final Integer workflowInstanceId = doBackfillWorkflow(
.commandParams(backfillWorkflowDTO.getStartParamList()) backfillWorkflowDTO,
.startNodes(backfillWorkflowDTO.getStartNodes()) backfillTimeList.stream().map(DateUtils::dateToString).collect(Collectors.toList()));
.backfillTimeList(backfillTimeList.stream().map(DateUtils::dateToString).collect(Collectors.toList())) return Lists.newArrayList(workflowInstanceId);
.timeZone(DateUtils.getTimezone())
.build();
doCreateCommand(backfillWorkflowDTO, backfillWorkflowCommandParam);
} }
private void doParallemBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) { private List<Integer> doParallemBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) {
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
Integer expectedParallelismNumber = backfillParams.getExpectedParallelismNumber(); Integer expectedParallelismNumber = backfillParams.getExpectedParallelismNumber();
@ -93,47 +96,60 @@ public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<Backf
} }
log.info("In parallel mode, current expectedParallelismNumber:{}", expectedParallelismNumber); log.info("In parallel mode, current expectedParallelismNumber:{}", expectedParallelismNumber);
final List<Integer> workflowInstanceIdList = Lists.newArrayList();
for (List<ZonedDateTime> stringDate : Lists.partition(listDate, expectedParallelismNumber)) { for (List<ZonedDateTime> stringDate : Lists.partition(listDate, expectedParallelismNumber)) {
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder() final Integer workflowInstanceId = doBackfillWorkflow(
.commandParams(backfillWorkflowDTO.getStartParamList()) backfillWorkflowDTO,
.startNodes(backfillWorkflowDTO.getStartNodes()) stringDate.stream().map(DateUtils::dateToString).collect(Collectors.toList()));
.backfillTimeList(stringDate.stream().map(DateUtils::dateToString).collect(Collectors.toList())) workflowInstanceIdList.add(workflowInstanceId);
.timeZone(DateUtils.getTimezone())
.build();
doCreateCommand(backfillWorkflowDTO, backfillWorkflowCommandParam);
} }
return workflowInstanceIdList;
} }
private void doCreateCommand(final BackfillWorkflowDTO backfillWorkflowDTO, private Integer doBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final BackfillWorkflowCommandParam backfillWorkflowCommandParam) { final List<String> backfillTimeList) {
List<String> backfillTimeList = backfillWorkflowCommandParam.getBackfillTimeList(); final Server masterServer = registryClient.getRandomServer(RegistryNodeType.MASTER).orElse(null);
final Command command = Command.builder() if (masterServer == null) {
.commandType(backfillWorkflowDTO.getExecType()) throw new ServiceException("no master server available");
.processDefinitionCode(backfillWorkflowDTO.getWorkflowDefinition().getCode()) }
.processDefinitionVersion(backfillWorkflowDTO.getWorkflowDefinition().getVersion())
.executorId(backfillWorkflowDTO.getLoginUser().getId()) final ProcessDefinition workflowDefinition = backfillWorkflowDTO.getWorkflowDefinition();
.scheduleTime(DateUtils.stringToDate(backfillTimeList.get(0))) final WorkflowBackfillTriggerRequest backfillTriggerRequest = WorkflowBackfillTriggerRequest.builder()
.commandParam(JSONUtils.toJsonString(backfillWorkflowCommandParam)) .userId(backfillWorkflowDTO.getLoginUser().getId())
.taskDependType(backfillWorkflowDTO.getTaskDependType()) .backfillTimeList(backfillTimeList)
.workflowCode(workflowDefinition.getCode())
.workflowVersion(workflowDefinition.getVersion())
.startNodes(backfillWorkflowDTO.getStartNodes())
.failureStrategy(backfillWorkflowDTO.getFailureStrategy()) .failureStrategy(backfillWorkflowDTO.getFailureStrategy())
.taskDependType(backfillWorkflowDTO.getTaskDependType())
.execType(backfillWorkflowDTO.getExecType())
.warningType(backfillWorkflowDTO.getWarningType()) .warningType(backfillWorkflowDTO.getWarningType())
.warningGroupId(backfillWorkflowDTO.getWarningGroupId()) .warningGroupId(backfillWorkflowDTO.getWarningGroupId())
.startTime(new Date()) .workflowInstancePriority(backfillWorkflowDTO.getWorkflowInstancePriority())
.processInstancePriority(backfillWorkflowDTO.getWorkflowInstancePriority())
.updateTime(new Date())
.workerGroup(backfillWorkflowDTO.getWorkerGroup()) .workerGroup(backfillWorkflowDTO.getWorkerGroup())
.tenantCode(backfillWorkflowDTO.getTenantCode()) .tenantCode(backfillWorkflowDTO.getTenantCode())
.dryRun(backfillWorkflowDTO.getDryRun().getCode()) .environmentCode(backfillWorkflowDTO.getEnvironmentCode())
.testFlag(backfillWorkflowDTO.getTestFlag().getCode()) .startParamList(backfillWorkflowDTO.getStartParamList())
.dryRun(backfillWorkflowDTO.getDryRun())
.testFlag(backfillWorkflowDTO.getTestFlag())
.build(); .build();
commandDao.insert(command);
final WorkflowBackfillTriggerResponse backfillTriggerResponse = Clients
.withService(IWorkflowControlClient.class)
.withHost(masterServer.getHost() + ":" + masterServer.getPort())
.backfillTriggerWorkflow(backfillTriggerRequest);
if (!backfillTriggerResponse.isSuccess()) {
throw new ServiceException("Backfill workflow failed: " + backfillTriggerResponse.getMessage());
}
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
if (backfillParams.getBackfillDependentMode() == ComplementDependentMode.ALL_DEPENDENT) { if (backfillParams.getBackfillDependentMode() == ComplementDependentMode.ALL_DEPENDENT) {
doBackfillDependentWorkflow(backfillWorkflowCommandParam, command); doBackfillDependentWorkflow(backfillWorkflowDTO, backfillTimeList);
} }
return backfillTriggerResponse.getWorkflowInstanceId();
} }
private void doBackfillDependentWorkflow(final BackfillWorkflowCommandParam backfillWorkflowCommandParam, private void doBackfillDependentWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final Command backfillCommand) { final List<String> backfillTimeList) {
// todo:
} }
} }

12
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/PauseWorkflowInstanceExecutorDelegate.java

@ -22,10 +22,10 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
@ -80,8 +80,8 @@ public class PauseWorkflowInstanceExecutorDelegate
private void pauseInMaster(ProcessInstance workflowInstance) { private void pauseInMaster(ProcessInstance workflowInstance) {
try { try {
final WorkflowInstancePauseResponse pauseResponse = SingletonJdkDynamicRpcClientProxyFactory final WorkflowInstancePauseResponse pauseResponse = Clients
.withService(IWorkflowInstanceController.class) .withService(IWorkflowControlClient.class)
.withHost(workflowInstance.getHost()) .withHost(workflowInstance.getHost())
.pauseWorkflowInstance(new WorkflowInstancePauseRequest(workflowInstance.getId())); .pauseWorkflowInstance(new WorkflowInstancePauseRequest(workflowInstance.getId()));

12
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/StopWorkflowInstanceExecutorDelegate.java

@ -22,10 +22,10 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
@ -81,8 +81,8 @@ public class StopWorkflowInstanceExecutorDelegate
void stopInMaster(ProcessInstance workflowInstance) { void stopInMaster(ProcessInstance workflowInstance) {
try { try {
final WorkflowInstanceStopResponse stopResponse = SingletonJdkDynamicRpcClientProxyFactory final WorkflowInstanceStopResponse stopResponse = Clients
.withService(IWorkflowInstanceController.class) .withService(IWorkflowControlClient.class)
.withHost(workflowInstance.getHost()) .withHost(workflowInstance.getHost())
.stopWorkflowInstance(new WorkflowInstanceStopRequest(workflowInstance.getId())); .stopWorkflowInstance(new WorkflowInstanceStopRequest(workflowInstance.getId()));

71
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/TriggerWorkflowExecutorDelegate.java

@ -17,55 +17,62 @@
package org.apache.dolphinscheduler.api.executor.workflow; package org.apache.dolphinscheduler.api.executor.workflow;
import org.apache.dolphinscheduler.api.exceptions.ServiceException;
import org.apache.dolphinscheduler.api.validator.workflow.TriggerWorkflowDTO; import org.apache.dolphinscheduler.api.validator.workflow.TriggerWorkflowDTO;
import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerResponse;
import org.apache.dolphinscheduler.service.process.TriggerRelationService; import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
import java.util.Date; import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component; import org.springframework.stereotype.Component;
@Slf4j
@Component @Component
public class TriggerWorkflowExecutorDelegate implements IExecutorDelegate<TriggerWorkflowDTO, Void> { public class TriggerWorkflowExecutorDelegate implements IExecutorDelegate<TriggerWorkflowDTO, Integer> {
@Autowired @Autowired
private CommandDao commandDao; private RegistryClient registryClient;
@Autowired
private TriggerRelationService triggerRelationService;
@Override @Override
public Void execute(TriggerWorkflowDTO triggerWorkflowDTO) { public Integer execute(final TriggerWorkflowDTO triggerWorkflowDTO) {
final RunWorkflowCommandParam runWorkflowCommandParam = final Server masterServer = registryClient.getRandomServer(RegistryNodeType.MASTER).orElse(null);
RunWorkflowCommandParam.builder() if (masterServer == null) {
.commandParams(triggerWorkflowDTO.getStartParamList()) throw new ServiceException("no master server available");
}
final WorkflowManualTriggerResponse workflowManualTriggerResponse = Clients
.withService(IWorkflowControlClient.class)
.withHost(masterServer.getHost() + ":" + masterServer.getPort())
.manualTriggerWorkflow(transform2WorkflowTriggerRequest(triggerWorkflowDTO));
if (!workflowManualTriggerResponse.isSuccess()) {
throw new ServiceException("Trigger workflow failed: " + workflowManualTriggerResponse.getMessage());
}
return workflowManualTriggerResponse.getWorkflowInstanceId();
}
private WorkflowManualTriggerRequest transform2WorkflowTriggerRequest(TriggerWorkflowDTO triggerWorkflowDTO) {
return WorkflowManualTriggerRequest.builder()
.userId(triggerWorkflowDTO.getLoginUser().getId())
.workflowDefinitionCode(triggerWorkflowDTO.getWorkflowDefinition().getCode())
.workflowDefinitionVersion(triggerWorkflowDTO.getWorkflowDefinition().getVersion())
.startNodes(triggerWorkflowDTO.getStartNodes()) .startNodes(triggerWorkflowDTO.getStartNodes())
.timeZone(DateUtils.getTimezone())
.build();
final Command command = Command.builder()
.commandType(triggerWorkflowDTO.getExecType())
.processDefinitionCode(triggerWorkflowDTO.getWorkflowDefinition().getCode())
.processDefinitionVersion(triggerWorkflowDTO.getWorkflowDefinition().getVersion())
.executorId(triggerWorkflowDTO.getLoginUser().getId())
.commandParam(JSONUtils.toJsonString(runWorkflowCommandParam))
.taskDependType(triggerWorkflowDTO.getTaskDependType())
.failureStrategy(triggerWorkflowDTO.getFailureStrategy()) .failureStrategy(triggerWorkflowDTO.getFailureStrategy())
.taskDependType(triggerWorkflowDTO.getTaskDependType())
.warningType(triggerWorkflowDTO.getWarningType()) .warningType(triggerWorkflowDTO.getWarningType())
.warningGroupId(triggerWorkflowDTO.getWarningGroupId()) .warningGroupId(triggerWorkflowDTO.getWarningGroupId())
.startTime(new Date()) .workflowInstancePriority(triggerWorkflowDTO.getWorkflowInstancePriority())
.processInstancePriority(triggerWorkflowDTO.getWorkflowInstancePriority())
.updateTime(new Date())
.workerGroup(triggerWorkflowDTO.getWorkerGroup()) .workerGroup(triggerWorkflowDTO.getWorkerGroup())
.tenantCode(triggerWorkflowDTO.getTenantCode()) .tenantCode(triggerWorkflowDTO.getTenantCode())
.dryRun(triggerWorkflowDTO.getDryRun().getCode()) .environmentCode(triggerWorkflowDTO.getEnvironmentCode())
.testFlag(triggerWorkflowDTO.getTestFlag().getCode()) .startParamList(triggerWorkflowDTO.getStartParamList())
.dryRun(triggerWorkflowDTO.getDryRun())
.testFlag(triggerWorkflowDTO.getTestFlag())
.build(); .build();
commandDao.insert(command);
return null;
} }
} }

9
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java

@ -25,19 +25,20 @@ import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.User;
import java.util.List;
import java.util.Map; import java.util.Map;
public interface ExecutorService { public interface ExecutorService {
/** /**
* Trigger the workflow and return the trigger code. * Trigger the workflow and return the workflow instance id.
*/ */
Long triggerWorkflowDefinition(final WorkflowTriggerRequest workflowTriggerRequest); Integer triggerWorkflowDefinition(final WorkflowTriggerRequest workflowTriggerRequest);
/** /**
* Backfill the workflow and return the trigger code. * Backfill the workflow and return the workflow instance ids.
*/ */
Long backfillWorkflowDefinition(final WorkflowBackFillRequest workflowBackFillRequest); List<Integer> backfillWorkflowDefinition(final WorkflowBackFillRequest workflowBackFillRequest);
/** /**
* check whether the process definition can be executed * check whether the process definition can be executed

9
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/AlertPluginInstanceServiceImpl.java

@ -39,7 +39,7 @@ import org.apache.dolphinscheduler.dao.mapper.PluginDefineMapper;
import org.apache.dolphinscheduler.extract.alert.IAlertOperator; import org.apache.dolphinscheduler.extract.alert.IAlertOperator;
import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse; import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse;
import org.apache.dolphinscheduler.extract.alert.request.AlertTestSendRequest; import org.apache.dolphinscheduler.extract.alert.request.AlertTestSendRequest;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.extract.base.utils.Host;
import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
@ -307,9 +307,10 @@ public class AlertPluginInstanceServiceImpl extends BaseServiceImpl implements A
AlertSendResponse alertSendResponse; AlertSendResponse alertSendResponse;
try { try {
IAlertOperator alertOperator = SingletonJdkDynamicRpcClientProxyFactory alertSendResponse = Clients
.getProxyClient(alertServerAddress.getAddress(), IAlertOperator.class); .withService(IAlertOperator.class)
alertSendResponse = alertOperator.sendTestAlert(alertTestSendRequest); .withHost(alertServerAddress.getAddress())
.sendTestAlert(alertTestSendRequest);
log.info("Send alert to: {} successfully, response: {}", alertServerAddress, alertSendResponse); log.info("Send alert to: {} successfully, response: {}", alertServerAddress, alertSendResponse);
} catch (Exception e) { } catch (Exception e) {
log.error("Send alert: {} to: {} failed", alertTestSendRequest, alertServerAddress, e); log.error("Send alert: {} to: {} failed", alertTestSendRequest, alertServerAddress, e);

17
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java

@ -81,7 +81,6 @@ import org.apache.dolphinscheduler.service.command.CommandService;
import org.apache.dolphinscheduler.service.cron.CronUtils; import org.apache.dolphinscheduler.service.cron.CronUtils;
import org.apache.dolphinscheduler.service.exceptions.CronParseException; import org.apache.dolphinscheduler.service.exceptions.CronParseException;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.process.TriggerRelationService;
import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.beanutils.BeanUtils;
import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.CollectionUtils;
@ -158,9 +157,6 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
@Autowired @Autowired
private WorkerGroupService workerGroupService; private WorkerGroupService workerGroupService;
@Autowired
private TriggerRelationService triggerRelationService;
@Autowired @Autowired
private TenantMapper tenantMapper; private TenantMapper tenantMapper;
@ -184,22 +180,19 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
@Override @Override
@Transactional @Transactional
public Long triggerWorkflowDefinition(final WorkflowTriggerRequest workflowTriggerRequest) { public Integer triggerWorkflowDefinition(final WorkflowTriggerRequest triggerRequest) {
final TriggerWorkflowDTO triggerWorkflowDTO = final TriggerWorkflowDTO triggerWorkflowDTO = triggerWorkflowRequestTransformer.transform(triggerRequest);
triggerWorkflowRequestTransformer.transform(workflowTriggerRequest);
triggerWorkflowDTOValidator.validate(triggerWorkflowDTO); triggerWorkflowDTOValidator.validate(triggerWorkflowDTO);
executorClient.triggerWorkflowDefinition().execute(triggerWorkflowDTO); return executorClient.triggerWorkflowDefinition().execute(triggerWorkflowDTO);
return triggerWorkflowDTO.getTriggerCode();
} }
@Override @Override
@Transactional @Transactional
public Long backfillWorkflowDefinition(final WorkflowBackFillRequest workflowBackFillRequest) { public List<Integer> backfillWorkflowDefinition(final WorkflowBackFillRequest workflowBackFillRequest) {
final BackfillWorkflowDTO backfillWorkflowDTO = final BackfillWorkflowDTO backfillWorkflowDTO =
backfillWorkflowRequestTransformer.transform(workflowBackFillRequest); backfillWorkflowRequestTransformer.transform(workflowBackFillRequest);
backfillWorkflowDTOValidator.validate(backfillWorkflowDTO); backfillWorkflowDTOValidator.validate(backfillWorkflowDTO);
executorClient.backfillWorkflowDefinition().execute(backfillWorkflowDTO); return executorClient.backfillWorkflowDefinition().execute(backfillWorkflowDTO);
return backfillWorkflowDTO.getTriggerCode();
} }
/** /**

21
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/LoggerServiceImpl.java

@ -34,7 +34,7 @@ import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.common.ILogService; import org.apache.dolphinscheduler.extract.common.ILogService;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadRequest; import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadRequest;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadResponse; import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadResponse;
@ -202,8 +202,6 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
sb.append(head); sb.append(head);
} }
ILogService iLogService =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(taskInstance.getHost(), ILogService.class);
try { try {
TaskInstanceLogPageQueryRequest request = TaskInstanceLogPageQueryRequest.builder() TaskInstanceLogPageQueryRequest request = TaskInstanceLogPageQueryRequest.builder()
.taskInstanceId(taskInstance.getId()) .taskInstanceId(taskInstance.getId())
@ -211,7 +209,10 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
.skipLineNum(skipLineNum) .skipLineNum(skipLineNum)
.limit(limit) .limit(limit)
.build(); .build();
TaskInstanceLogPageQueryResponse response = iLogService.pageQueryTaskInstanceLog(request); final TaskInstanceLogPageQueryResponse response = Clients
.withService(ILogService.class)
.withHost(taskInstance.getHost())
.pageQueryTaskInstanceLog(request);
String logContent = response.getLogContent(); String logContent = response.getLogContent();
if (logContent != null) { if (logContent != null) {
sb.append(logContent); sb.append(logContent);
@ -239,12 +240,14 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
byte[] logBytes; byte[] logBytes;
ILogService iLogService =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(taskInstance.getHost(), ILogService.class);
try { try {
TaskInstanceLogFileDownloadRequest request = final TaskInstanceLogFileDownloadRequest request = new TaskInstanceLogFileDownloadRequest(
new TaskInstanceLogFileDownloadRequest(taskInstance.getId(), logPath); taskInstance.getId(),
TaskInstanceLogFileDownloadResponse response = iLogService.getTaskInstanceWholeLogFileBytes(request); logPath);
final TaskInstanceLogFileDownloadResponse response = Clients
.withService(ILogService.class)
.withHost(taskInstance.getHost())
.getTaskInstanceWholeLogFileBytes(request);
logBytes = response.getLogBytes(); logBytes = response.getLogBytes();
return Bytes.concat(head, logBytes); return Bytes.concat(head, logBytes);
} catch (Exception ex) { } catch (Exception ex) {

9
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/MetricsCleanUpServiceImpl.java

@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.api.service.impl;
import org.apache.dolphinscheduler.api.metrics.ApiServerMetrics; import org.apache.dolphinscheduler.api.metrics.ApiServerMetrics;
import org.apache.dolphinscheduler.api.service.MetricsCleanUpService; import org.apache.dolphinscheduler.api.service.MetricsCleanUpService;
import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceService; import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceService;
import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
@ -49,10 +49,9 @@ public class MetricsCleanUpServiceImpl implements MetricsCleanUpService {
private void cleanUpWorkflowMetrics(Server server, Long workflowDefinitionCode) { private void cleanUpWorkflowMetrics(Server server, Long workflowDefinitionCode) {
try { try {
IWorkflowInstanceService iWorkflowInstanceService = Clients.withService(IWorkflowInstanceService.class)
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient( .withHost(server.getHost() + ":" + server.getPort())
String.format("%s:%s", server.getHost(), server.getPort()), IWorkflowInstanceService.class); .clearWorkflowMetrics(workflowDefinitionCode);
iWorkflowInstanceService.clearWorkflowMetrics(workflowDefinitionCode);
} catch (Exception e) { } catch (Exception e) {
log.error( log.error(
"Fail to clean up workflow related metrics on {} when deleting workflow definition {}, error message {}", "Fail to clean up workflow related metrics on {} when deleting workflow definition {}, error message {}",

27
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java

@ -45,7 +45,7 @@ import org.apache.dolphinscheduler.dao.repository.DqExecuteResultDao;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils; import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.common.ILogService; import org.apache.dolphinscheduler.extract.common.ILogService;
import org.apache.dolphinscheduler.extract.worker.IStreamingTaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.IStreamingTaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
@ -277,11 +277,10 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
putMsg(result, Status.TASK_INSTANCE_NOT_FOUND); putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
return result; return result;
} }
IStreamingTaskInstanceOperator streamingTaskInstanceOperator = final TaskInstanceTriggerSavepointResponse taskInstanceTriggerSavepointResponse = Clients
SingletonJdkDynamicRpcClientProxyFactory .withService(IStreamingTaskInstanceOperator.class)
.getProxyClient(taskInstance.getHost(), IStreamingTaskInstanceOperator.class); .withHost(taskInstance.getHost())
TaskInstanceTriggerSavepointResponse taskInstanceTriggerSavepointResponse = .triggerSavepoint(new TaskInstanceTriggerSavepointRequest(taskInstanceId));
streamingTaskInstanceOperator.triggerSavepoint(new TaskInstanceTriggerSavepointRequest(taskInstanceId));
log.info("StreamingTaskInstance trigger savepoint response: {}", taskInstanceTriggerSavepointResponse); log.info("StreamingTaskInstance trigger savepoint response: {}", taskInstanceTriggerSavepointResponse);
putMsg(result, Status.SUCCESS); putMsg(result, Status.SUCCESS);
return result; return result;
@ -310,10 +309,10 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
} }
// todo: we only support streaming task for now // todo: we only support streaming task for now
ITaskInstanceOperator iTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory final TaskInstanceKillResponse taskInstanceKillResponse = Clients
.getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class); .withService(ITaskInstanceOperator.class)
TaskInstanceKillResponse taskInstanceKillResponse = .withHost(taskInstance.getHost())
iTaskInstanceOperator.killTask(new TaskInstanceKillRequest(taskInstanceId)); .killTask(new TaskInstanceKillRequest(taskInstanceId));
log.info("TaskInstance kill response: {}", taskInstanceKillResponse); log.info("TaskInstance kill response: {}", taskInstanceKillResponse);
putMsg(result, Status.SUCCESS); putMsg(result, Status.SUCCESS);
@ -369,10 +368,10 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
if (StringUtils.isNotBlank(taskInstance.getLogPath())) { if (StringUtils.isNotBlank(taskInstance.getLogPath())) {
try { try {
// Remove task instance log failed will not affect the deletion of task instance // Remove task instance log failed will not affect the deletion of task instance
ILogService iLogService = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(taskInstance.getHost(), .withService(ILogService.class)
ILogService.class); .withHost(taskInstance.getHost())
iLogService.removeTaskInstanceLog(taskInstance.getLogPath()); .removeTaskInstanceLog(taskInstance.getLogPath());
} catch (Exception ex) { } catch (Exception ex) {
log.error("Remove task instance log error", ex); log.error("Remove task instance log error", ex);
} }

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/validator/workflow/TriggerWorkflowDTO.java

@ -73,6 +73,4 @@ public class TriggerWorkflowDTO {
private Flag testFlag; private Flag testFlag;
private Long triggerCode;
} }

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/validator/workflow/TriggerWorkflowRequestTransformer.java

@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.api.dto.workflow.WorkflowTriggerRequest;
import org.apache.dolphinscheduler.api.exceptions.ServiceException; import org.apache.dolphinscheduler.api.exceptions.ServiceException;
import org.apache.dolphinscheduler.api.utils.WorkflowUtils; import org.apache.dolphinscheduler.api.utils.WorkflowUtils;
import org.apache.dolphinscheduler.api.validator.ITransformer; import org.apache.dolphinscheduler.api.validator.ITransformer;
import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao;
import org.apache.dolphinscheduler.plugin.task.api.utils.PropertyUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.PropertyUtils;
@ -56,7 +55,6 @@ public class TriggerWorkflowRequestTransformer implements ITransformer<WorkflowT
PropertyUtils.startParamsTransformPropertyList(workflowTriggerRequest.getStartParamList())) PropertyUtils.startParamsTransformPropertyList(workflowTriggerRequest.getStartParamList()))
.dryRun(workflowTriggerRequest.getDryRun()) .dryRun(workflowTriggerRequest.getDryRun())
.testFlag(workflowTriggerRequest.getTestFlag()) .testFlag(workflowTriggerRequest.getTestFlag())
.triggerCode(CodeGenerateUtils.genCode())
.build(); .build();
ProcessDefinition workflowDefinition = processDefinitionDao ProcessDefinition workflowDefinition = processDefinitionDao

10
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowInstanceExecuteFunctionControllerTest.java

@ -89,7 +89,7 @@ public class WorkflowInstanceExecuteFunctionControllerTest extends AbstractContr
final Integer version = 1; final Integer version = 1;
final boolean allLevelDependent = false; final boolean allLevelDependent = false;
final JsonObject expectResponseContent = gson final JsonObject expectResponseContent = gson
.fromJson("{\"code\":0,\"msg\":\"success\",\"data\":1,\"success\":true,\"failed\":false}", .fromJson("{\"code\":0,\"msg\":\"success\",\"data\":[1],\"success\":true,\"failed\":false}",
JsonObject.class); JsonObject.class);
final ImmutableMap<String, Object> executeServiceResult = final ImmutableMap<String, Object> executeServiceResult =
@ -123,7 +123,7 @@ public class WorkflowInstanceExecuteFunctionControllerTest extends AbstractContr
paramsMap.add("executionOrder", String.valueOf(executionOrder)); paramsMap.add("executionOrder", String.valueOf(executionOrder));
paramsMap.add("version", String.valueOf(version)); paramsMap.add("version", String.valueOf(version));
when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1L); when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1);
// When // When
final MvcResult mvcResult = mockMvc final MvcResult mvcResult = mockMvc
@ -163,7 +163,7 @@ public class WorkflowInstanceExecuteFunctionControllerTest extends AbstractContr
paramsMap.add("executionOrder", String.valueOf(executionOrder)); paramsMap.add("executionOrder", String.valueOf(executionOrder));
paramsMap.add("version", String.valueOf(version)); paramsMap.add("version", String.valueOf(version));
when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1L); when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1);
// When // When
final MvcResult mvcResult = mockMvc final MvcResult mvcResult = mockMvc
@ -203,7 +203,7 @@ public class WorkflowInstanceExecuteFunctionControllerTest extends AbstractContr
paramsMap.add("executionOrder", String.valueOf(executionOrder)); paramsMap.add("executionOrder", String.valueOf(executionOrder));
paramsMap.add("version", String.valueOf(version)); paramsMap.add("version", String.valueOf(version));
when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1L); when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1);
// When // When
final MvcResult mvcResult = mockMvc final MvcResult mvcResult = mockMvc
@ -229,7 +229,7 @@ public class WorkflowInstanceExecuteFunctionControllerTest extends AbstractContr
paramsMap.add("scheduleTime", scheduleTime); paramsMap.add("scheduleTime", scheduleTime);
paramsMap.add("version", String.valueOf(version)); paramsMap.add("version", String.valueOf(version));
when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1L); when(executorService.triggerWorkflowDefinition(Mockito.any())).thenReturn(1);
final MvcResult mvcResult = mockMvc final MvcResult mvcResult = mockMvc
.perform(post("/projects/{projectCode}/executors/start-process-instance", projectCode) .perform(post("/projects/{projectCode}/executors/start-process-instance", projectCode)

1
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java

@ -22,6 +22,7 @@ import java.util.Map;
import com.baomidou.mybatisplus.annotation.EnumValue; import com.baomidou.mybatisplus.annotation.EnumValue;
// todo: rename to WorkflowTriggerType
public enum CommandType { public enum CommandType {
/** /**

2
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkflowExecutionStatus.java

@ -26,6 +26,7 @@ import com.baomidou.mybatisplus.annotation.EnumValue;
public enum WorkflowExecutionStatus { public enum WorkflowExecutionStatus {
SUBMITTED_SUCCESS(0, "submitted"),
RUNNING_EXECUTION(1, "running"), RUNNING_EXECUTION(1, "running"),
READY_PAUSE(2, "ready pause"), READY_PAUSE(2, "ready pause"),
PAUSE(3, "pause"), PAUSE(3, "pause"),
@ -45,6 +46,7 @@ public enum WorkflowExecutionStatus {
}; };
private static final int[] NOT_TERMINAL_STATUS = new int[]{ private static final int[] NOT_TERMINAL_STATUS = new int[]{
SUBMITTED_SUCCESS.getCode(),
RUNNING_EXECUTION.getCode(), RUNNING_EXECUTION.getCode(),
READY_PAUSE.getCode(), READY_PAUSE.getCode(),
READY_STOP.getCode(), READY_STOP.getCode(),

35
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Command.java

@ -51,8 +51,11 @@ public class Command {
@TableField("process_definition_code") @TableField("process_definition_code")
private long processDefinitionCode; private long processDefinitionCode;
@TableField("executor_id") @TableField("process_definition_version")
private int executorId; private int processDefinitionVersion;
@TableField("process_instance_id")
private int processInstanceId;
/** /**
* command parameter, format json * command parameter, format json
@ -60,56 +63,66 @@ public class Command {
@TableField("command_param") @TableField("command_param")
private String commandParam; private String commandParam;
@TableField("process_instance_priority")
private Priority processInstancePriority;
@Deprecated
@TableField("executor_id")
private int executorId;
@Deprecated
@TableField("task_depend_type") @TableField("task_depend_type")
@Builder.Default @Builder.Default
private TaskDependType taskDependType = TaskDependType.TASK_POST; private TaskDependType taskDependType = TaskDependType.TASK_POST;
@Deprecated
@TableField("failure_strategy") @TableField("failure_strategy")
@Builder.Default @Builder.Default
private FailureStrategy failureStrategy = FailureStrategy.CONTINUE; private FailureStrategy failureStrategy = FailureStrategy.CONTINUE;
@Deprecated
@TableField("warning_type") @TableField("warning_type")
private WarningType warningType; private WarningType warningType;
@Deprecated
@TableField("warning_group_id") @TableField("warning_group_id")
private Integer warningGroupId; private Integer warningGroupId;
@Deprecated
@TableField("schedule_time") @TableField("schedule_time")
private Date scheduleTime; private Date scheduleTime;
@Deprecated
@TableField("start_time") @TableField("start_time")
private Date startTime = new Date(); private Date startTime = new Date();
@TableField("process_instance_priority") @Deprecated
private Priority processInstancePriority;
@TableField("update_time") @TableField("update_time")
@Builder.Default @Builder.Default
private Date updateTime = new Date(); private Date updateTime = new Date();
@Deprecated
@TableField("worker_group") @TableField("worker_group")
private String workerGroup; private String workerGroup;
/** /**
* tenant code * tenant code
*/ */
@Deprecated
private String tenantCode; private String tenantCode;
@Deprecated
@TableField("environment_code") @TableField("environment_code")
private Long environmentCode; private Long environmentCode;
@Deprecated
@TableField("dry_run") @TableField("dry_run")
private int dryRun; private int dryRun;
@TableField("process_instance_id")
private int processInstanceId;
@TableField("process_definition_version")
private int processDefinitionVersion;
/** /**
* test flag * test flag
*/ */
@Deprecated
@TableField("test_flag") @TableField("test_flag")
private int testFlag; private int testFlag;

63
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TriggerRelation.java

@ -1,63 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import java.util.Date;
import lombok.Data;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
@Data
@TableName("t_ds_trigger_relation")
public class TriggerRelation {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private Integer id;
/**
* trigger code
*/
private long triggerCode;
/**
* triggerType
*/
private int triggerType;
/**
* jobId
*/
private Integer jobId;
/**
* create time
*/
private Date createTime;
/**
* update time
*/
private Date updateTime;
}

72
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapper.java

@ -1,72 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.TriggerRelation;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* triggerRelation mapper interface
*/
public interface TriggerRelationMapper extends BaseMapper<TriggerRelation> {
/**
* query by code and id
* @param triggerType
* @param jobId
* @return
*/
List<TriggerRelation> queryByTypeAndJobId(@Param("triggerType") Integer triggerType, @Param("jobId") int jobId);
/**
* query triggerRelation by code
*
* @param triggerCode triggerCode
* @return triggerRelation
*/
List<TriggerRelation> queryByTriggerRelationCode(@Param("triggerCode") Long triggerCode);
/**
* query triggerRelation by code
*
* @param triggerCode triggerCode
* @return triggerRelation
*/
List<TriggerRelation> queryByTriggerRelationCodeAndType(@Param("triggerCode") Long triggerCode,
@Param("triggerType") Integer triggerType);
/**
* delete triggerRelation by code
*
* @param triggerCode triggerCode
* @return int
*/
int deleteByCode(@Param("triggerCode") Long triggerCode);
/**
* if exist update else insert
*
* @param triggerRelation
*/
void upsert(@Param("triggerRelation") TriggerRelation triggerRelation);
}

23
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/UserDao.java

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.repository;
import org.apache.dolphinscheduler.dao.entity.User;
public interface UserDao extends IDao<User> {
}

36
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/UserDaoImpl.java

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.repository.impl;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.UserMapper;
import org.apache.dolphinscheduler.dao.repository.BaseDao;
import org.apache.dolphinscheduler.dao.repository.UserDao;
import lombok.NonNull;
import org.springframework.stereotype.Repository;
@Repository
public class UserDaoImpl extends BaseDao<User, UserMapper> implements UserDao {
public UserDaoImpl(@NonNull UserMapper userMapper) {
super(userMapper);
}
}

61
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapper.xml

@ -1,61 +0,0 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.TriggerRelationMapper">
<sql id="baseSql">
id, trigger_code, trigger_type, job_id, create_time, update_time
</sql>
<select id="queryByTypeAndJobId" resultType="org.apache.dolphinscheduler.dao.entity.TriggerRelation">
select
<include refid="baseSql"/>
from t_ds_trigger_relation
WHERE trigger_type = #{triggerType} and job_id = #{jobId}
</select>
<select id="queryByTriggerRelationCode" resultType="org.apache.dolphinscheduler.dao.entity.TriggerRelation">
select
<include refid="baseSql"/>
from t_ds_trigger_relation
WHERE trigger_code = #{triggerCode}
</select>
<select id="queryByTriggerRelationCodeAndType" resultType="org.apache.dolphinscheduler.dao.entity.TriggerRelation">
select
<include refid="baseSql"/>
from t_ds_trigger_relation
WHERE trigger_code = #{triggerCode} and trigger_type = #{triggerType}
</select>
<delete id="deleteByCode">
delete from t_ds_trigger_relation where triggerCode = #{triggerCode}
</delete>
<insert id="upsert">
INSERT INTO t_ds_trigger_relation (trigger_code, trigger_type, job_id, create_time, update_time) VALUES(
#{triggerRelation.triggerCode},#{triggerRelation.triggerType},#{triggerRelation.jobId},#{triggerRelation.createTime},#{triggerRelation.updateTime})
ON DUPLICATE KEY UPDATE update_time = #{triggerRelation.updateTime};
</insert>
<insert id="upsert" databaseId="PostgreSQL">
INSERT INTO t_ds_trigger_relation (trigger_code, trigger_type, job_id, create_time, update_time) VALUES(
#{triggerRelation.triggerCode},#{triggerRelation.triggerType},#{triggerRelation.jobId},#{triggerRelation.createTime},#{triggerRelation.updateTime})
ON CONFLICT(trigger_type,job_id,trigger_code) DO UPDATE SET update_time = #{triggerRelation.updateTime};
</insert>
</mapper>

16
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql

@ -2128,22 +2128,6 @@ CREATE TABLE t_ds_fav_task
PRIMARY KEY (id) PRIMARY KEY (id)
); );
--
-- Table structure for t_ds_trigger_relation
--
DROP TABLE IF EXISTS `t_ds_trigger_relation`;
CREATE TABLE t_ds_trigger_relation
(
id bigint(20) NOT NULL AUTO_INCREMENT,
trigger_type int NOT NULL,
job_id int NOT NULL,
trigger_code bigint(20) NOT NULL,
create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
PRIMARY KEY (id),
UNIQUE KEY t_ds_trigger_relation_UN(trigger_type,job_id,trigger_code)
);
DROP TABLE IF EXISTS t_ds_relation_sub_workflow; DROP TABLE IF EXISTS t_ds_relation_sub_workflow;
CREATE TABLE t_ds_relation_sub_workflow ( CREATE TABLE t_ds_relation_sub_workflow (

16
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql

@ -2107,22 +2107,6 @@ CREATE TABLE `t_ds_fav_task`
AUTO_INCREMENT = 1 AUTO_INCREMENT = 1
DEFAULT CHARSET = utf8 COLLATE = utf8_bin; DEFAULT CHARSET = utf8 COLLATE = utf8_bin;
-- ----------------------------
-- Table structure for t_ds_trigger_relation
-- ----------------------------
DROP TABLE IF EXISTS `t_ds_trigger_relation`;
CREATE TABLE `t_ds_trigger_relation` (
`id` bigint(20) NOT NULL AUTO_INCREMENT,
`trigger_type` int(11) NOT NULL DEFAULT '0' COMMENT '0 process 1 task',
`trigger_code` bigint(20) NOT NULL,
`job_id` bigint(20) NOT NULL,
`create_time` datetime DEFAULT NULL,
`update_time` datetime DEFAULT NULL,
PRIMARY KEY (`id`),
KEY `t_ds_trigger_relation_trigger_code_IDX` (`trigger_code`),
UNIQUE KEY `t_ds_trigger_relation_UN` (`trigger_type`,`job_id`,`trigger_code`)
) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8 COLLATE = utf8_bin;
DROP TABLE IF EXISTS `t_ds_relation_sub_workflow`; DROP TABLE IF EXISTS `t_ds_relation_sub_workflow`;
CREATE TABLE `t_ds_relation_sub_workflow` ( CREATE TABLE `t_ds_relation_sub_workflow` (

15
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql

@ -2098,21 +2098,6 @@ CREATE TABLE t_ds_fav_task
PRIMARY KEY (id) PRIMARY KEY (id)
); );
-- ----------------------------
-- Table structure for t_ds_trigger_relation
-- ----------------------------
DROP TABLE IF EXISTS t_ds_trigger_relation;
CREATE TABLE t_ds_trigger_relation (
id serial NOT NULL,
trigger_type int NOT NULL,
trigger_code bigint NOT NULL,
job_id bigint NOT NULL,
create_time timestamp DEFAULT NULL,
update_time timestamp DEFAULT NULL,
PRIMARY KEY (id),
CONSTRAINT t_ds_trigger_relation_unique UNIQUE (trigger_type,job_id,trigger_code)
);
DROP TABLE IF EXISTS t_ds_relation_sub_workflow; DROP TABLE IF EXISTS t_ds_relation_sub_workflow;
CREATE TABLE t_ds_relation_sub_workflow ( CREATE TABLE t_ds_relation_sub_workflow (
id serial NOT NULL, id serial NOT NULL,

2
dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql

@ -125,3 +125,5 @@ d//
delimiter ; delimiter ;
CALL drop_column_t_ds_alert_plugin_instance; CALL drop_column_t_ds_alert_plugin_instance;
DROP PROCEDURE drop_column_t_ds_alert_plugin_instance; DROP PROCEDURE drop_column_t_ds_alert_plugin_instance;
DROP TABLE IF EXISTS `t_ds_trigger_relation`;

2
dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql

@ -123,3 +123,5 @@ d//
select drop_column_t_ds_alert_plugin_instance(); select drop_column_t_ds_alert_plugin_instance();
DROP FUNCTION IF EXISTS drop_column_t_ds_alert_plugin_instance(); DROP FUNCTION IF EXISTS drop_column_t_ds_alert_plugin_instance();
DROP TABLE IF EXISTS t_ds_trigger_relation;

134
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TriggerRelationMapperTest.java

@ -1,134 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import static com.google.common.truth.Truth.assertThat;
import org.apache.dolphinscheduler.common.enums.ApiTriggerType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.BaseDaoTest;
import org.apache.dolphinscheduler.dao.entity.TriggerRelation;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
/**
* trigger mapper test
*/
public class TriggerRelationMapperTest extends BaseDaoTest {
@Autowired
TriggerRelationMapper triggerRelationMapper;
/**
* test insert
*
* @return
*/
@Test
public void testInsert() {
TriggerRelation expectedObj = createTriggerRelation();
Assertions.assertTrue(expectedObj.getId() > 0);
}
/**
* test select by id
*
* @return
*/
@Test
public void testSelectById() {
TriggerRelation expectRelation = createTriggerRelation();
TriggerRelation actualRelation = triggerRelationMapper.selectById(expectRelation.getId());
Assertions.assertEquals(expectRelation, actualRelation);
}
/**
* test select by type and job id
*
* @return
*/
@Test
public void testQueryByTypeAndJobId() {
TriggerRelation expectRelation = createTriggerRelation();
assertThat(
triggerRelationMapper.queryByTypeAndJobId(expectRelation.getTriggerType(), expectRelation.getJobId()))
.containsExactly(expectRelation);
}
/**
* test select by trigger code
*
* @return
*/
@Test
public void testQueryByTriggerRelationCode() {
TriggerRelation expectRelation = createTriggerRelation();
assertThat(triggerRelationMapper.queryByTriggerRelationCode(expectRelation.getTriggerCode()))
.containsExactly(expectRelation);
}
/**
* test select by type and trigger code
*
* @return
*/
@Test
public void testQueryByTriggerRelationCodeAndType() {
TriggerRelation expectRelation = createTriggerRelation();
assertThat(triggerRelationMapper.queryByTriggerRelationCodeAndType(expectRelation.getTriggerCode(),
expectRelation.getTriggerType())).containsExactly(expectRelation);
}
@Test
public void testUpsert() {
TriggerRelation expectRelation = createTriggerRelation();
triggerRelationMapper.upsert(expectRelation);
assertThat(triggerRelationMapper.selectById(expectRelation.getId())).isEqualTo(expectRelation);
}
/**
* test delete
*/
@Test
public void testDelete() {
TriggerRelation expectRelation = createTriggerRelation();
triggerRelationMapper.deleteById(expectRelation.getId());
assertThat(triggerRelationMapper.selectById(expectRelation.getId())).isNull();
}
/**
* create TriggerRelation and insert
*
* @return TriggerRelation
* @throws Exception
*/
private TriggerRelation createTriggerRelation() {
TriggerRelation triggerRelation = new TriggerRelation();
triggerRelation.setTriggerCode(4567890);
triggerRelation.setTriggerType(ApiTriggerType.COMMAND.getCode());
triggerRelation.setJobId(99);
triggerRelation.setCreateTime(DateUtils.getCurrentDate());
triggerRelation.setUpdateTime(DateUtils.getCurrentDate());
triggerRelationMapper.insert(triggerRelation);
return triggerRelation;
}
}

22
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactory.java → dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/Clients.java

@ -22,15 +22,21 @@ import org.apache.dolphinscheduler.extract.base.config.NettyClientConfig;
import lombok.AccessLevel; import lombok.AccessLevel;
import lombok.NoArgsConstructor; import lombok.NoArgsConstructor;
/**
* The factory class for creating a dynamic proxy client.
* <pre>
* final IService proxyClient = Clients
* .withService(IService.class)
* .withHost(serverAddress);
* </pre>
*/
@NoArgsConstructor(access = AccessLevel.PRIVATE) @NoArgsConstructor(access = AccessLevel.PRIVATE)
public class SingletonJdkDynamicRpcClientProxyFactory { public class Clients {
private static final JdkDynamicRpcClientProxyFactory INSTANCE = new JdkDynamicRpcClientProxyFactory(
NettyRemotingClientFactory.buildNettyRemotingClient(new NettyClientConfig()));
public static <T> T getProxyClient(String serverAddress, Class<T> clazz) { private static final JdkDynamicRpcClientProxyFactory jdkDynamicRpcClientProxyFactory =
return INSTANCE.getProxyClient(serverAddress, clazz); new JdkDynamicRpcClientProxyFactory(
} NettyRemotingClientFactory.buildNettyRemotingClient(
new NettyClientConfig()));
public static <T> JdkDynamicRpcClientProxyBuilder<T> withService(Class<T> serviceClazz) { public static <T> JdkDynamicRpcClientProxyBuilder<T> withService(Class<T> serviceClazz) {
return new JdkDynamicRpcClientProxyBuilder<>(serviceClazz); return new JdkDynamicRpcClientProxyBuilder<>(serviceClazz);
@ -45,7 +51,7 @@ public class SingletonJdkDynamicRpcClientProxyFactory {
} }
public T withHost(String serviceHost) { public T withHost(String serviceHost) {
return getProxyClient(serviceHost, serviceClazz); return jdkDynamicRpcClientProxyFactory.getProxyClient(serviceHost, serviceClazz);
} }
} }

17
dolphinscheduler-extract/dolphinscheduler-extract-base/src/test/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactoryTest.java → dolphinscheduler-extract/dolphinscheduler-extract-base/src/test/java/org/apache/dolphinscheduler/extract/base/client/ClientsTest.java

@ -34,7 +34,7 @@ import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
public class SingletonJdkDynamicRpcClientProxyFactoryTest { public class ClientsTest {
private SpringServerMethodInvokerDiscovery springServerMethodInvokerDiscovery; private SpringServerMethodInvokerDiscovery springServerMethodInvokerDiscovery;
@ -55,15 +55,17 @@ public class SingletonJdkDynamicRpcClientProxyFactoryTest {
@Test @Test
public void getProxyClient() { public void getProxyClient() {
IService proxyClient = IService proxyClient = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(serverAddress, IService.class); .withService(IService.class)
.withHost(serverAddress);
Assertions.assertNotNull(proxyClient); Assertions.assertNotNull(proxyClient);
} }
@Test @Test
public void testPing() { public void testPing() {
IService proxyClient = IService proxyClient = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(serverAddress, IService.class); .withService(IService.class)
.withHost(serverAddress);
assertEquals("pong", proxyClient.ping("ping")); assertEquals("pong", proxyClient.ping("ping"));
MethodInvocationException methodInvocationException = MethodInvocationException methodInvocationException =
@ -73,8 +75,9 @@ public class SingletonJdkDynamicRpcClientProxyFactoryTest {
@Test @Test
public void testVoid() { public void testVoid() {
IService proxyClient = IService proxyClient = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(serverAddress, IService.class); .withService(IService.class)
.withHost(serverAddress);
assertDoesNotThrow(proxyClient::voidMethod); assertDoesNotThrow(proxyClient::voidMethod);
} }

53
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowControlClient.java

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerResponse;
/**
* Workflow instance controller used to do control operation for workflow instance.
*/
@RpcService
public interface IWorkflowControlClient {
@RpcMethod
WorkflowManualTriggerResponse manualTriggerWorkflow(final WorkflowManualTriggerRequest workflowManualTriggerRequest);
@RpcMethod
WorkflowBackfillTriggerResponse backfillTriggerWorkflow(final WorkflowBackfillTriggerRequest workflowBackfillTriggerRequest);
@RpcMethod
WorkflowScheduleTriggerResponse scheduleTriggerWorkflow(final WorkflowScheduleTriggerRequest workflowScheduleTriggerRequest);
@RpcMethod
WorkflowInstancePauseResponse pauseWorkflowInstance(final WorkflowInstancePauseRequest workflowInstancePauseRequest);
@RpcMethod
WorkflowInstanceStopResponse stopWorkflowInstance(final WorkflowInstanceStopRequest workflowInstanceStopRequest);
}

39
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowInstanceController.java

@ -1,39 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse;
/**
* Workflow instance controller used to do control operation for workflow instance.
*/
@RpcService
public interface IWorkflowInstanceController {
@RpcMethod
WorkflowInstancePauseResponse pauseWorkflowInstance(WorkflowInstancePauseRequest workflowInstancePauseRequest);
@RpcMethod
WorkflowInstanceStopResponse stopWorkflowInstance(WorkflowInstanceStopRequest workflowInstanceStopRequest);
}

83
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowBackfillTriggerRequest.java

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import java.util.ArrayList;
import java.util.List;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowBackfillTriggerRequest {
private Integer userId;
private List<String> backfillTimeList;
private Long workflowCode;
private Integer workflowVersion;
private List<Long> startNodes;
@Builder.Default
private FailureStrategy failureStrategy = FailureStrategy.CONTINUE;
@Builder.Default
private TaskDependType taskDependType = TaskDependType.TASK_POST;
private CommandType execType;
@Builder.Default
private WarningType warningType = WarningType.NONE;
private Integer warningGroupId;
@Builder.Default
private Priority workflowInstancePriority = Priority.MEDIUM;
private String workerGroup;
private String tenantCode;
private Long environmentCode;
@Builder.Default
private List<Property> startParamList = new ArrayList<>();
@Builder.Default
private Flag dryRun = Flag.NO;
@Builder.Default
private Flag testFlag = Flag.NO;
}

51
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowBackfillTriggerResponse.java

@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowBackfillTriggerResponse {
private boolean success;
private String message;
private Integer workflowInstanceId;
public static WorkflowBackfillTriggerResponse fail(String message) {
return WorkflowBackfillTriggerResponse.builder()
.success(false)
.message(message)
.build();
}
public static WorkflowBackfillTriggerResponse success(Integer workflowInstanceId) {
return WorkflowBackfillTriggerResponse.builder()
.success(true)
.workflowInstanceId(workflowInstanceId)
.build();
}
}

2
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/WorkflowInstancePauseRequest.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstancePauseRequest.java

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.extract.master.transportor; package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Data; import lombok.Data;

2
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/WorkflowInstancePauseResponse.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstancePauseResponse.java

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.extract.master.transportor; package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Data; import lombok.Data;

2
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/WorkflowInstanceStopRequest.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstanceStopRequest.java

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.extract.master.transportor; package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Data; import lombok.Data;

2
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/WorkflowInstanceStopResponse.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowInstanceStopResponse.java

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.extract.master.transportor; package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Data; import lombok.Data;

77
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowManualTriggerRequest.java

@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import java.util.ArrayList;
import java.util.List;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowManualTriggerRequest {
private Integer userId;
private Long workflowDefinitionCode;
private Integer workflowDefinitionVersion;
private List<Long> startNodes;
@Builder.Default
private FailureStrategy failureStrategy = FailureStrategy.CONTINUE;
@Builder.Default
private TaskDependType taskDependType = TaskDependType.TASK_POST;
@Builder.Default
private WarningType warningType = WarningType.NONE;
private Integer warningGroupId;
@Builder.Default
private Priority workflowInstancePriority = Priority.MEDIUM;
private String workerGroup;
private String tenantCode;
private Long environmentCode;
@Builder.Default
private List<Property> startParamList = new ArrayList<>();
@Builder.Default
private Flag dryRun = Flag.NO;
@Builder.Default
private Flag testFlag = Flag.NO;
}

50
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowManualTriggerResponse.java

@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowManualTriggerResponse {
private boolean success;
private String message;
private Integer workflowInstanceId;
public static WorkflowManualTriggerResponse fail(String message) {
return WorkflowManualTriggerResponse.builder()
.success(false)
.message(message)
.build();
}
public static WorkflowManualTriggerResponse success(Integer workflowInstanceId) {
return WorkflowManualTriggerResponse.builder()
.success(true)
.workflowInstanceId(workflowInstanceId)
.build();
}
}

75
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowScheduleTriggerRequest.java

@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import java.util.Date;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowScheduleTriggerRequest {
private Integer userId;
private Date scheduleTIme;
private String timezoneId;
private Long workflowCode;
private Integer workflowVersion;
@Builder.Default
private FailureStrategy failureStrategy = FailureStrategy.CONTINUE;
@Builder.Default
private TaskDependType taskDependType = TaskDependType.TASK_POST;
@Builder.Default
private WarningType warningType = WarningType.NONE;
private Integer warningGroupId;
@Builder.Default
private Priority workflowInstancePriority = Priority.MEDIUM;
private String workerGroup;
private String tenantCode;
private Long environmentCode;
@Builder.Default
private Flag dryRun = Flag.NO;
@Builder.Default
private Flag testFlag = Flag.NO;
}

51
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/workflow/WorkflowScheduleTriggerResponse.java

@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor.workflow;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class WorkflowScheduleTriggerResponse {
private boolean success;
private String message;
private Integer workflowInstanceId;
public static WorkflowScheduleTriggerResponse fail(String message) {
return WorkflowScheduleTriggerResponse.builder()
.success(false)
.message(message)
.build();
}
public static WorkflowScheduleTriggerResponse success(Integer workflowInstanceId) {
return WorkflowScheduleTriggerResponse.builder()
.success(true)
.workflowInstanceId(workflowInstanceId)
.build();
}
}

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

@ -32,7 +32,7 @@ import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskGroupDao; import org.apache.dolphinscheduler.dao.repository.TaskGroupDao;
import org.apache.dolphinscheduler.dao.repository.TaskGroupQueueDao; import org.apache.dolphinscheduler.dao.repository.TaskGroupQueueDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskInstanceController; import org.apache.dolphinscheduler.extract.master.ITaskInstanceController;
import org.apache.dolphinscheduler.extract.master.transportor.TaskGroupSlotAcquireSuccessNotifyRequest; import org.apache.dolphinscheduler.extract.master.transportor.TaskGroupSlotAcquireSuccessNotifyRequest;
import org.apache.dolphinscheduler.extract.master.transportor.TaskGroupSlotAcquireSuccessNotifyResponse; import org.apache.dolphinscheduler.extract.master.transportor.TaskGroupSlotAcquireSuccessNotifyResponse;
@ -471,7 +471,7 @@ public class TaskGroupCoordinator extends BaseDaemonThread {
.build(); .build();
TaskGroupSlotAcquireSuccessNotifyResponse taskGroupSlotAcquireSuccessNotifyResponse = TaskGroupSlotAcquireSuccessNotifyResponse taskGroupSlotAcquireSuccessNotifyResponse =
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceController.class) .withService(ITaskInstanceController.class)
.withHost(processInstance.getHost()) .withHost(processInstance.getHost())
.notifyTaskGroupSlotAcquireSuccess(taskGroupSlotAcquireSuccessNotifyRequest); .notifyTaskGroupSlotAcquireSuccess(taskGroupSlotAcquireSuccessNotifyRequest);

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

@ -33,6 +33,7 @@ import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.config.MasterServerLoadProtection; import org.apache.dolphinscheduler.server.master.config.MasterServerLoadProtection;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBusCoordinator; import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBusCoordinator;
import org.apache.dolphinscheduler.server.master.engine.exceptions.CommandDuplicateHandleException;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStartLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStartLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.WorkflowExecutionRunnableFactory; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.WorkflowExecutionRunnableFactory;
@ -177,6 +178,12 @@ public class CommandEngine extends BaseDaemonThread implements AutoCloseable {
} }
private Void bootstrapError(Command command, Throwable throwable) { private Void bootstrapError(Command command, Throwable throwable) {
if (throwable instanceof CommandDuplicateHandleException) {
log.warn("Handle command failed, the command: {} has been handled by other master",
command,
throwable);
return null;
}
log.error("Failed bootstrap command {} ", JSONUtils.toPrettyJsonString(command), throwable); log.error("Failed bootstrap command {} ", JSONUtils.toPrettyJsonString(command), throwable);
commandService.moveToErrorCommand(command, ExceptionUtils.getStackTrace(throwable)); commandService.moveToErrorCommand(command, ExceptionUtils.getStackTrace(throwable));
return null; return null;

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

@ -40,7 +40,6 @@ import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.W
import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -67,9 +66,7 @@ public abstract class AbstractCommandHandler implements ICommandHandler {
@Override @Override
public WorkflowExecutionRunnable handleCommand(final Command command) { public WorkflowExecutionRunnable handleCommand(final Command command) {
final WorkflowExecuteContextBuilder workflowExecuteContextBuilder = WorkflowExecuteContext final WorkflowExecuteContextBuilder workflowExecuteContextBuilder = WorkflowExecuteContext.builder()
.builder()
.withWorkflowInstanceLifecycleListeners(workflowLifecycleListeners)
.withCommand(command); .withCommand(command);
assembleWorkflowDefinition(workflowExecuteContextBuilder); assembleWorkflowDefinition(workflowExecuteContextBuilder);
@ -94,7 +91,8 @@ public abstract class AbstractCommandHandler implements ICommandHandler {
protected void assembleWorkflowInstanceLifecycleListeners( protected void assembleWorkflowInstanceLifecycleListeners(
final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) { final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) {
workflowExecuteContextBuilder.setWorkflowInstanceLifecycleListeners(new ArrayList<>()); workflowExecuteContextBuilder.setWorkflowInstanceLifecycleListeners(
workflowExecuteContextBuilder.getWorkflowInstanceLifecycleListeners());
} }
protected void assembleWorkflowDefinition( protected void assembleWorkflowDefinition(

21
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/BackfillWorkflowCommandHandler.java

@ -18,14 +18,8 @@
package org.apache.dolphinscheduler.server.master.engine.command.handler; package org.apache.dolphinscheduler.server.master.engine.command.handler;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.workflow.listener.WorkflowSuccessLifecycleListener;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.ApplicationContext;
import org.springframework.stereotype.Component; import org.springframework.stereotype.Component;
/** /**
@ -35,21 +29,6 @@ import org.springframework.stereotype.Component;
@Component @Component
public class BackfillWorkflowCommandHandler extends RunWorkflowCommandHandler { public class BackfillWorkflowCommandHandler extends RunWorkflowCommandHandler {
@Autowired
private ProcessInstanceDao processInstanceDao;
@Autowired
private TaskInstanceDao taskInstanceDao;
@Autowired
private MasterConfig masterConfig;
@Autowired
private ApplicationContext applicationContext;
@Autowired
private WorkflowSuccessLifecycleListener workflowSuccessLifecycleListener;
@Override @Override
public CommandType commandType() { public CommandType commandType() {
return CommandType.COMPLEMENT_DATA; return CommandType.COMPLEMENT_DATA;

5
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/handler/ReRunWorkflowCommandHandler.java

@ -24,6 +24,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
import java.util.Date; import java.util.Date;
@ -48,6 +49,9 @@ public class ReRunWorkflowCommandHandler extends RunWorkflowCommandHandler {
@Autowired @Autowired
private ApplicationContext applicationContext; private ApplicationContext applicationContext;
@Autowired
private MasterConfig masterConfig;
/** /**
* Generate the repeat running workflow instance. * Generate the repeat running workflow instance.
* <p> Will use the origin workflow instance, but will update the following fields. Need to note we cannot not * <p> Will use the origin workflow instance, but will update the following fields. Need to note we cannot not
@ -71,6 +75,7 @@ public class ReRunWorkflowCommandHandler extends RunWorkflowCommandHandler {
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name()); workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());
workflowInstance.setCommandType(command.getCommandType()); workflowInstance.setCommandType(command.getCommandType());
workflowInstance.setRestartTime(new Date()); workflowInstance.setRestartTime(new Date());
workflowInstance.setHost(masterConfig.getMasterAddress());
workflowInstance.setEndTime(null); workflowInstance.setEndTime(null);
workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1); workflowInstance.setRunTimes(workflowInstance.getRunTimes() + 1);
workflowInstanceDao.updateById(workflowInstance); workflowInstanceDao.updateById(workflowInstance);

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

@ -18,18 +18,13 @@
package org.apache.dolphinscheduler.server.master.engine.command.handler; package org.apache.dolphinscheduler.server.master.engine.command.handler;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.master.command.ICommandParam; import org.apache.dolphinscheduler.extract.master.command.ICommandParam;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.model.Property; import org.apache.dolphinscheduler.plugin.task.api.model.Property;
@ -43,9 +38,7 @@ import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.W
import org.apache.dolphinscheduler.service.expand.CuringParamsService; import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.ObjectUtils;
import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -86,45 +79,15 @@ public class RunWorkflowCommandHandler extends AbstractCommandHandler {
protected void assembleWorkflowInstance(final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) { protected void assembleWorkflowInstance(final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) {
final ProcessDefinition workflowDefinition = workflowExecuteContextBuilder.getWorkflowDefinition(); final ProcessDefinition workflowDefinition = workflowExecuteContextBuilder.getWorkflowDefinition();
final Command command = workflowExecuteContextBuilder.getCommand(); final Command command = workflowExecuteContextBuilder.getCommand();
final ProcessInstance workflowInstance = processInstanceDao.queryById(command.getProcessInstanceId());
final ProcessInstance processInstance = new ProcessInstance(); workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());
processInstance.setProcessDefinitionCode(workflowDefinition.getCode()); workflowInstance.setHost(masterConfig.getMasterAddress());
processInstance.setProcessDefinitionVersion(workflowDefinition.getVersion()); workflowInstance.setCommandParam(command.getCommandParam());
processInstance.setProjectCode(workflowDefinition.getProjectCode()); workflowInstance.setGlobalParams(mergeCommandParamsWithWorkflowParams(command, workflowDefinition));
processInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name()); processInstanceDao.upsertProcessInstance(workflowInstance);
processInstance.setRecovery(Flag.NO);
processInstance.setStartTime(new Date()); workflowExecuteContextBuilder.setWorkflowInstance(workflowInstance);
processInstance.setRestartTime(processInstance.getStartTime());
processInstance.setEndTime(null);
processInstance.setRunTimes(1);
processInstance.setName(String.join("-", workflowDefinition.getName(), DateUtils.getCurrentTimeStamp()));
processInstance.setHost(masterConfig.getMasterAddress());
processInstance.setCommandType(command.getCommandType());
processInstance.setCommandParam(command.getCommandParam());
processInstance.setTaskDependType(command.getTaskDependType());
processInstance.setFailureStrategy(command.getFailureStrategy());
processInstance.setWarningType(ObjectUtils.defaultIfNull(command.getWarningType(), WarningType.NONE));
processInstance.setWarningGroupId(command.getWarningGroupId());
processInstance.setScheduleTime(command.getScheduleTime());
// todo: merge the global params or add startup params
// or can we merge this after
processInstance.setGlobalParams(mergeCommandParamsWithWorkflowParams(command, workflowDefinition));
processInstance.setExecutorId(command.getExecutorId());
processInstance.setExecutorName(null);
processInstance.setTenantCode(command.getTenantCode());
processInstance.setIsSubProcess(Flag.NO);
processInstance.addHistoryCmd(command.getCommandType());
processInstance.setProcessInstancePriority(command.getProcessInstancePriority());
processInstance.setWorkerGroup(WorkerGroupUtils.getWorkerGroupOrDefault(command.getWorkerGroup()));
processInstance.setEnvironmentCode(EnvironmentUtils.getEnvironmentCodeOrDefault(command.getEnvironmentCode()));
processInstance.setTimeout(workflowDefinition.getTimeout());
processInstance.setVarPool(null);
processInstance.setDryRun(command.getDryRun());
processInstance.setTestFlag(command.getTestFlag());
processInstanceDao.insert(processInstance);
workflowExecuteContextBuilder.setWorkflowInstance(processInstance);
} }
@Override @Override

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

@ -18,13 +18,8 @@
package org.apache.dolphinscheduler.server.master.engine.command.handler; package org.apache.dolphinscheduler.server.master.engine.command.handler;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.scheduler.api.SchedulerApi; import org.apache.dolphinscheduler.scheduler.api.SchedulerApi;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.ApplicationContext;
import org.springframework.stereotype.Component; import org.springframework.stereotype.Component;
/** /**
@ -33,18 +28,6 @@ import org.springframework.stereotype.Component;
@Component @Component
public class ScheduleWorkflowCommandHandler extends RunWorkflowCommandHandler { public class ScheduleWorkflowCommandHandler extends RunWorkflowCommandHandler {
@Autowired
private ProcessInstanceDao processInstanceDao;
@Autowired
private TaskInstanceDao taskInstanceDao;
@Autowired
private MasterConfig masterConfig;
@Autowired
private ApplicationContext applicationContext;
@Override @Override
public CommandType commandType() { public CommandType commandType() {
return CommandType.SCHEDULER; return CommandType.SCHEDULER;

6
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java

@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.server.master.engine.task.client;
import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator; import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest; import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse; import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse;
@ -51,7 +51,7 @@ public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDele
final String taskName = taskInstance.getName(); final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final LogicTaskPauseResponse pauseResponse = SingletonJdkDynamicRpcClientProxyFactory final LogicTaskPauseResponse pauseResponse = Clients
.withService(ILogicTaskInstanceOperator.class) .withService(ILogicTaskInstanceOperator.class)
.withHost(taskInstance.getHost()) .withHost(taskInstance.getHost())
.pauseLogicTask(new LogicTaskPauseRequest(taskInstance.getId())); .pauseLogicTask(new LogicTaskPauseRequest(taskInstance.getId()));
@ -69,7 +69,7 @@ public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDele
final String taskName = taskInstance.getName(); final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final LogicTaskKillResponse killResponse = SingletonJdkDynamicRpcClientProxyFactory final LogicTaskKillResponse killResponse = Clients
.withService(ILogicTaskInstanceOperator.class) .withService(ILogicTaskInstanceOperator.class)
.withHost(taskInstance.getHost()) .withHost(taskInstance.getHost())
.killLogicTask(new LogicTaskKillRequest(taskInstance.getId())); .killLogicTask(new LogicTaskKillRequest(taskInstance.getId()));

6
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java

@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.server.master.engine.task.client;
import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse;
@ -50,7 +50,7 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD
final String taskName = taskInstance.getName(); final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final TaskInstancePauseResponse pauseResponse = SingletonJdkDynamicRpcClientProxyFactory final TaskInstancePauseResponse pauseResponse = Clients
.withService(ITaskInstanceOperator.class) .withService(ITaskInstanceOperator.class)
.withHost(taskInstance.getHost()) .withHost(taskInstance.getHost())
.pauseTask(new TaskInstancePauseRequest(taskInstance.getId())); .pauseTask(new TaskInstancePauseRequest(taskInstance.getId()));
@ -68,7 +68,7 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD
final String taskName = taskInstance.getName(); final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final TaskInstanceKillResponse killResponse = SingletonJdkDynamicRpcClientProxyFactory final TaskInstanceKillResponse killResponse = Clients
.withService(ITaskInstanceOperator.class) .withService(ITaskInstanceOperator.class)
.withHost(executorHost) .withHost(executorHost)
.killTask(new TaskInstanceKillRequest(taskInstance.getId())); .killTask(new TaskInstanceKillRequest(taskInstance.getId()));

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java

@ -23,7 +23,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionSta
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskRequest; import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskResponse; import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskResponse;
@ -256,7 +256,7 @@ public abstract class AbstractTaskStateAction implements ITaskStateAction {
.taskInstanceId(taskExecutionRunnable.getTaskInstance().getId()) .taskInstanceId(taskExecutionRunnable.getTaskInstance().getId())
.workflowHost(masterConfig.getMasterAddress()) .workflowHost(masterConfig.getMasterAddress())
.build(); .build();
final TakeOverTaskResponse takeOverTaskResponse = SingletonJdkDynamicRpcClientProxyFactory final TakeOverTaskResponse takeOverTaskResponse = Clients
.withService(ITaskInstanceOperator.class) .withService(ITaskInstanceOperator.class)
.withHost(taskExecutionRunnable.getTaskInstance().getHost()) .withHost(taskExecutionRunnable.getTaskInstance().getHost())
.takeOverTask(takeOverTaskRequest); .takeOverTask(takeOverTaskRequest);

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

@ -21,14 +21,16 @@ import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.command.ICommandParam; import org.apache.dolphinscheduler.extract.master.command.ICommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.AbstractWorkflowLifecycleLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.AbstractWorkflowLifecycleLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.WorkflowLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.WorkflowLifecycleEventType;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowBackfillTrigger;
import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.CollectionUtils;
@ -43,6 +45,9 @@ import org.springframework.stereotype.Component;
@Component @Component
public class WorkflowSuccessLifecycleListener implements IWorkflowLifecycleListener { public class WorkflowSuccessLifecycleListener implements IWorkflowLifecycleListener {
@Autowired
private WorkflowBackfillTrigger workflowBackfillTrigger;
@Autowired @Autowired
private CommandDao commandDao; private CommandDao commandDao;
@ -76,30 +81,30 @@ public class WorkflowSuccessLifecycleListener implements IWorkflowLifecycleListe
if (CollectionUtils.isEmpty(backfillTimeList)) { if (CollectionUtils.isEmpty(backfillTimeList)) {
return; return;
} }
final BackfillWorkflowCommandParam nextCommandParam = BackfillWorkflowCommandParam.builder() final WorkflowBackfillTriggerRequest backfillTriggerRequest = WorkflowBackfillTriggerRequest.builder()
.userId(workflowInstance.getExecutorId())
.backfillTimeList(backfillTimeList) .backfillTimeList(backfillTimeList)
.commandParams(commandParam.getCommandParams()) .workflowCode(workflowInstance.getProcessDefinitionCode())
.timeZone(commandParam.getTimeZone()) .workflowVersion(workflowInstance.getProcessDefinitionVersion())
.build(); .startNodes(commandParam.getStartNodes())
final Command command = Command.builder()
.commandType(CommandType.COMPLEMENT_DATA)
.processDefinitionCode(workflowInstance.getProcessDefinitionCode())
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion())
.executorId(workflowInstance.getExecutorId())
.failureStrategy(workflowInstance.getFailureStrategy()) .failureStrategy(workflowInstance.getFailureStrategy())
.taskDependType(workflowInstance.getTaskDependType()) .taskDependType(workflowInstance.getTaskDependType())
.commandParam(JSONUtils.toJsonString(nextCommandParam)) .execType(CommandType.COMPLEMENT_DATA)
.scheduleTime(DateUtils.stringToDate(backfillTimeList.get(0)))
.warningType(workflowInstance.getWarningType()) .warningType(workflowInstance.getWarningType())
.warningGroupId(workflowInstance.getWarningGroupId()) .warningGroupId(workflowInstance.getWarningGroupId())
.workflowInstancePriority(workflowInstance.getProcessInstancePriority())
.workerGroup(workflowInstance.getWorkerGroup()) .workerGroup(workflowInstance.getWorkerGroup())
.tenantCode(workflowInstance.getTenantCode()) .tenantCode(workflowInstance.getTenantCode())
.environmentCode(workflowInstance.getEnvironmentCode()) .environmentCode(workflowInstance.getEnvironmentCode())
.processInstancePriority(workflowInstance.getProcessInstancePriority()) .startParamList(commandParam.getCommandParams())
.dryRun(Flag.of(workflowInstance.getDryRun()))
.testFlag(Flag.of(workflowInstance.getTestFlag()))
.build(); .build();
final WorkflowBackfillTriggerResponse backfillTriggerResponse =
commandDao.insert(command); workflowBackfillTrigger.triggerWorkflow(backfillTriggerRequest);
if (!backfillTriggerResponse.isSuccess()) {
log.warn("Backfill workflow failed: {}", backfillTriggerResponse.getMessage());
}
} }
@Override @Override

59
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/IWorkflowExecutionRunnable.java

@ -31,34 +31,75 @@ public interface IWorkflowExecutionRunnable {
/** /**
* Get the id of the WorkflowExecutionRunnable. * Get the id of the WorkflowExecutionRunnable.
*/ */
int getId(); default int getId() {
return getWorkflowInstance().getId();
}
/** /**
* Get the name of the WorkflowExecutionRunnable. * Get the name of the WorkflowExecutionRunnable.
*/ */
String getName(); default String getName() {
return getWorkflowInstance().getName();
}
/**
* Pause the WorkflowExecutionRunnable.
*/
void pause();
/**
* Whether the workflow is ready to pause.
*/
default boolean isWorkflowReadyPause() {
final WorkflowExecutionStatus workflowExecutionStatus = getWorkflowInstance().getState();
return workflowExecutionStatus == WorkflowExecutionStatus.READY_PAUSE;
}
/**
* Stop the WorkflowExecutionRunnable.
*/
void stop();
/**
* Whether the workflow is ready to stop.
*/
default boolean isWorkflowReadyStop() {
final WorkflowExecutionStatus workflowExecutionStatus = getWorkflowInstance().getState();
return workflowExecutionStatus == WorkflowExecutionStatus.READY_STOP;
}
/**
* Get the WorkflowExecuteContext belongs to the WorkflowExecutionRunnable.
*/
IWorkflowExecuteContext getWorkflowExecuteContext(); IWorkflowExecuteContext getWorkflowExecuteContext();
/** /**
* Get the WorkflowInstance belongs to the WorkflowExecutionRunnable. * Get the WorkflowInstance belongs to the WorkflowExecutionRunnable.
*/ */
ProcessInstance getWorkflowInstance(); default ProcessInstance getWorkflowInstance() {
return getWorkflowExecuteContext().getWorkflowInstance();
}
/** /**
* Get the state of the WorkflowExecutionRunnable. * Get the state of the WorkflowExecutionRunnable.
*/ */
WorkflowExecutionStatus getState(); default WorkflowExecutionStatus getState() {
return getWorkflowInstance().getState();
}
/** /**
* Get the WorkflowEventBus belongs to the Workflow instance. * Get the WorkflowEventBus belongs to the Workflow instance.
*/ */
WorkflowEventBus getWorkflowEventBus(); default WorkflowEventBus getWorkflowEventBus() {
return getWorkflowExecuteContext().getWorkflowEventBus();
}
/** /**
* Get the WorkflowExecutionGraph belongs to the Workflow instance. * Get the WorkflowExecutionGraph belongs to the Workflow instance.
*/ */
IWorkflowExecutionGraph getWorkflowExecutionGraph(); default IWorkflowExecutionGraph getWorkflowExecutionGraph() {
return getWorkflowExecuteContext().getWorkflowExecutionGraph();
}
/** /**
* Get the WorkflowInstanceLifecycleListeners belongs to the Workflow instance. * Get the WorkflowInstanceLifecycleListeners belongs to the Workflow instance.
@ -66,12 +107,8 @@ public interface IWorkflowExecutionRunnable {
List<IWorkflowLifecycleListener> getWorkflowLifecycleListeners(); List<IWorkflowLifecycleListener> getWorkflowLifecycleListeners();
/** /**
* Whether the workflow is ready to pause. * Register a WorkflowInstanceLifecycleListener to the Workflow instance.
*/ */
boolean isWorkflowReadyPause();
boolean isWorkflowReadyStop();
void registerWorkflowInstanceLifecycleListener(IWorkflowLifecycleListener listener); void registerWorkflowInstanceLifecycleListener(IWorkflowLifecycleListener listener);
} }

50
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/WorkflowExecutionRunnable.java

@ -17,10 +17,10 @@
package org.apache.dolphinscheduler.server.master.engine.workflow.runnable; package org.apache.dolphinscheduler.server.master.engine.workflow.runnable;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowPauseLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowExecutionGraph; import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStopLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.listener.IWorkflowLifecycleListener; import org.apache.dolphinscheduler.server.master.engine.workflow.listener.IWorkflowLifecycleListener;
import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext; import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext;
@ -37,56 +37,23 @@ public class WorkflowExecutionRunnable implements IWorkflowExecutionRunnable {
@Getter @Getter
private final IWorkflowExecuteContext workflowExecuteContext; private final IWorkflowExecuteContext workflowExecuteContext;
private final ProcessInstance workflowInstance;
@Getter
private final IWorkflowExecutionGraph workflowExecutionGraph;
@Getter
private final WorkflowEventBus workflowEventBus;
@Getter @Getter
private final List<IWorkflowLifecycleListener> workflowInstanceLifecycleListeners; private final List<IWorkflowLifecycleListener> workflowInstanceLifecycleListeners;
public WorkflowExecutionRunnable(WorkflowExecutionRunnableBuilder workflowExecutionRunnableBuilder) { public WorkflowExecutionRunnable(WorkflowExecutionRunnableBuilder workflowExecutionRunnableBuilder) {
final ApplicationContext applicationContext = workflowExecutionRunnableBuilder.getApplicationContext(); final ApplicationContext applicationContext = workflowExecutionRunnableBuilder.getApplicationContext();
this.workflowExecuteContext = workflowExecutionRunnableBuilder.getWorkflowExecuteContextBuilder().build(); this.workflowExecuteContext = workflowExecutionRunnableBuilder.getWorkflowExecuteContextBuilder().build();
this.workflowInstance = workflowExecuteContext.getWorkflowInstance();
this.workflowExecutionGraph = workflowExecuteContext.getWorkflowExecutionGraph();
this.workflowEventBus = workflowExecuteContext.getWorkflowEventBus();
this.workflowInstanceLifecycleListeners = workflowExecuteContext.getWorkflowInstanceLifecycleListeners(); this.workflowInstanceLifecycleListeners = workflowExecuteContext.getWorkflowInstanceLifecycleListeners();
} }
@Override @Override
public int getId() { public void pause() {
return workflowInstance.getId(); getWorkflowEventBus().publish(WorkflowPauseLifecycleEvent.of(this));
}
@Override
public String getName() {
return workflowInstance.getName();
}
@Override
public boolean isWorkflowReadyPause() {
final WorkflowExecutionStatus workflowExecutionStatus = workflowInstance.getState();
return workflowExecutionStatus == WorkflowExecutionStatus.READY_PAUSE;
}
@Override
public boolean isWorkflowReadyStop() {
final WorkflowExecutionStatus workflowExecutionStatus = workflowInstance.getState();
return workflowExecutionStatus == WorkflowExecutionStatus.READY_STOP;
}
@Override
public ProcessInstance getWorkflowInstance() {
return workflowExecuteContext.getWorkflowInstance();
} }
@Override @Override
public WorkflowExecutionStatus getState() { public void stop() {
return workflowInstance.getState(); getWorkflowEventBus().publish(WorkflowStopLifecycleEvent.of(this));
} }
@Override @Override
@ -96,6 +63,7 @@ public class WorkflowExecutionRunnable implements IWorkflowExecutionRunnable {
@Override @Override
public void registerWorkflowInstanceLifecycleListener(IWorkflowLifecycleListener listener) { public void registerWorkflowInstanceLifecycleListener(IWorkflowLifecycleListener listener) {
checkArgument(listener != null, "listener cannot be null");
workflowInstanceLifecycleListeners.add(listener); workflowInstanceLifecycleListeners.add(listener);
} }

8
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/runnable/WorkflowExecutionRunnableFactory.java

@ -23,7 +23,6 @@ import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.server.master.engine.command.ICommandHandler; import org.apache.dolphinscheduler.server.master.engine.command.ICommandHandler;
import org.apache.dolphinscheduler.server.master.engine.exceptions.CommandDuplicateHandleException; import org.apache.dolphinscheduler.server.master.engine.exceptions.CommandDuplicateHandleException;
import org.apache.dolphinscheduler.server.master.engine.workflow.listener.IWorkflowLifecycleListener;
import java.util.List; import java.util.List;
@ -46,9 +45,6 @@ public class WorkflowExecutionRunnableFactory {
@Autowired @Autowired
private CommandDao commandDao; private CommandDao commandDao;
@Autowired
private List<IWorkflowLifecycleListener> workflowLifecycleListeners;
/** /**
* Generate WorkflowExecutionRunnable from command. * Generate WorkflowExecutionRunnable from command.
* <p> We use transaction here to make sure that the command will be handled only once. Since in some case if the * <p> We use transaction here to make sure that the command will be handled only once. Since in some case if the
@ -57,9 +53,7 @@ public class WorkflowExecutionRunnableFactory {
@Transactional @Transactional
public IWorkflowExecutionRunnable createWorkflowExecuteRunnable(Command command) { public IWorkflowExecutionRunnable createWorkflowExecuteRunnable(Command command) {
deleteCommandOrThrow(command); deleteCommandOrThrow(command);
final IWorkflowExecutionRunnable workflowExecutionRunnable = doCreateWorkflowExecutionRunnable(command); return doCreateWorkflowExecutionRunnable(command);
workflowLifecycleListeners.forEach(workflowExecutionRunnable::registerWorkflowInstanceLifecycleListener);
return workflowExecutionRunnable;
} }
/** /**

1
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/IWorkflowStateAction.java

@ -33,6 +33,7 @@ import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkf
* Represents the action to be taken when a workflow is in a certain state and receive a target event. * Represents the action to be taken when a workflow is in a certain state and receive a target event.
* <p> Each {@link WorkflowExecutionStatus} should have a corresponding {@link IWorkflowStateAction} implementation. * <p> Each {@link WorkflowExecutionStatus} should have a corresponding {@link IWorkflowStateAction} implementation.
* *
* @see WorkflowSubmittedStateAction
* @see WorkflowRunningStateAction * @see WorkflowRunningStateAction
* @see WorkflowReadyPauseStateAction * @see WorkflowReadyPauseStateAction
* @see WorkflowPausedStateAction * @see WorkflowPausedStateAction

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowStateActionFactory.java

@ -18,7 +18,6 @@
package org.apache.dolphinscheduler.server.master.engine.workflow.statemachine; package org.apache.dolphinscheduler.server.master.engine.workflow.statemachine;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.server.master.engine.WorkflowCacheRepository;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
@ -32,8 +31,7 @@ public class WorkflowStateActionFactory {
private final Map<WorkflowExecutionStatus, IWorkflowStateAction> workflowStateActionMap = new HashMap<>(); private final Map<WorkflowExecutionStatus, IWorkflowStateAction> workflowStateActionMap = new HashMap<>();
public WorkflowStateActionFactory(List<IWorkflowStateAction> workflowStateActions, public WorkflowStateActionFactory(List<IWorkflowStateAction> workflowStateActions) {
WorkflowCacheRepository workflowCacheRepository) {
workflowStateActions.forEach( workflowStateActions.forEach(
workflowStateAction -> workflowStateActionMap.put(workflowStateAction.matchState(), workflowStateAction -> workflowStateActionMap.put(workflowStateAction.matchState(),
workflowStateAction)); workflowStateAction));

121
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/statemachine/WorkflowSubmittedStateAction.java

@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.statemachine;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowFailedLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowFinalizeLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowPauseLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowPausedLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStartLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStopLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStoppedLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowSucceedLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
/**
* The workflow submitted state should transform to running state when handle command.
*/
@Slf4j
@Component
public class WorkflowSubmittedStateAction extends AbstractWorkflowStateAction {
@Override
public void startEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowStartLifecycleEvent workflowStartEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowStartEvent);
}
@Override
public void topologyLogicalTransitionEventAction(
final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent workflowTopologyLogicalTransitionWithTaskFinishEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowTopologyLogicalTransitionWithTaskFinishEvent);
}
@Override
public void pauseEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowPauseLifecycleEvent workflowPauseEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowPauseEvent);
}
@Override
public void pausedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowPausedLifecycleEvent workflowPausedEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowPausedEvent);
}
@Override
public void stopEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowStopLifecycleEvent workflowStopEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowStopEvent);
}
@Override
public void stoppedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowStoppedLifecycleEvent workflowStoppedEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowStoppedEvent);
}
@Override
public void succeedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowSucceedLifecycleEvent workflowSucceedEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowSucceedEvent);
}
@Override
public void failedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowFailedLifecycleEvent workflowFailedEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowFailedEvent);
}
@Override
public void finalizeEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final WorkflowFinalizeLifecycleEvent workflowFinalizeEvent) {
throwExceptionIfStateIsNotMatch(workflowExecutionRunnable);
logWarningIfCannotDoAction(workflowExecutionRunnable, workflowFinalizeEvent);
}
@Override
public WorkflowExecutionStatus matchState() {
return WorkflowExecutionStatus.SUBMITTED_SUCCESS;
}
/**
* The running state can only finish with success/failure.
*/
@Override
protected void emitWorkflowFinishedEventIfApplicable(IWorkflowExecutionRunnable workflowExecutionRunnable) {
throw new IllegalStateException(
"The workflow " + workflowExecutionRunnable.getName() +
"is submitted, shouldn't emit workflow finished event");
}
}

86
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/AbstractWorkflowTrigger.java

@ -0,0 +1,86 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionLogDao;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.UserDao;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.transaction.annotation.Transactional;
@Slf4j
public abstract class AbstractWorkflowTrigger<TriggerRequest, TriggerResponse>
implements
IWorkflowTrigger<TriggerRequest, TriggerResponse> {
@Autowired
private ProcessDefinitionLogDao workflowDefinitionDao;
@Autowired
private ProcessInstanceDao workflowInstanceDao;
@Autowired
private UserDao userDao;
@Autowired
private CommandDao commandDao;
@Override
@Transactional
public TriggerResponse triggerWorkflow(final TriggerRequest triggerRequest) {
final ProcessInstance workflowInstance = constructWorkflowInstance(triggerRequest);
workflowInstanceDao.insert(workflowInstance);
final Command command = constructTriggerCommand(triggerRequest, workflowInstance);
commandDao.insert(command);
return onTriggerSuccess(workflowInstance);
}
protected abstract ProcessInstance constructWorkflowInstance(final TriggerRequest triggerRequest);
protected abstract Command constructTriggerCommand(final TriggerRequest triggerRequest,
final ProcessInstance workflowInstance);
protected abstract TriggerResponse onTriggerSuccess(final ProcessInstance workflowInstance);
protected ProcessDefinition getProcessDefinition(final Long workflowCode, final Integer workflowVersion) {
final ProcessDefinitionLog workflow = workflowDefinitionDao.queryByDefinitionCodeAndVersion(
workflowCode, workflowVersion);
if (workflow == null) {
throw new IllegalStateException(
"Workflow definition not found: " + workflowCode + " version " + workflowVersion);
}
return workflow;
}
protected User getExecutorUser(final Integer userId) {
return userDao.queryOptionalById(userId)
.orElseThrow(() -> new IllegalStateException("User not found: " + userId));
}
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/IWorkflowTrigger.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
/**
* The trigger interface of the workflow, used to trigger the workflow and generate the workflow instance.
*
* @param <TriggerRequest> trigger request
* @param <TriggerResponse> trigger response
*/
public interface IWorkflowTrigger<TriggerRequest, TriggerResponse> {
TriggerResponse triggerWorkflow(final TriggerRequest triggerRequest);
}

114
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowBackfillTrigger.java

@ -0,0 +1,114 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.commons.lang3.ObjectUtils;
import java.util.Date;
import java.util.List;
import org.springframework.stereotype.Component;
/**
* Backfill trigger of the workflow, used to trigger the workflow and generate the workflow instance in the backfill way.
*/
@Component
public class WorkflowBackfillTrigger
extends
AbstractWorkflowTrigger<WorkflowBackfillTriggerRequest, WorkflowBackfillTriggerResponse> {
@Override
protected ProcessInstance constructWorkflowInstance(WorkflowBackfillTriggerRequest backfillTriggerRequest) {
final CommandType commandType = CommandType.COMPLEMENT_DATA;
final Long workflowCode = backfillTriggerRequest.getWorkflowCode();
final Integer workflowVersion = backfillTriggerRequest.getWorkflowVersion();
final List<String> backfillTimeList = backfillTriggerRequest.getBackfillTimeList();
final ProcessDefinition workflowDefinition = getProcessDefinition(workflowCode, workflowVersion);
final ProcessInstance workflowInstance = new ProcessInstance();
workflowInstance.setProcessDefinitionCode(workflowDefinition.getCode());
workflowInstance.setProcessDefinitionVersion(workflowDefinition.getVersion());
workflowInstance.setProjectCode(workflowDefinition.getProjectCode());
workflowInstance.setCommandType(commandType);
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.SUBMITTED_SUCCESS, commandType.name());
workflowInstance.setRecovery(Flag.NO);
workflowInstance.setScheduleTime(DateUtils.stringToDate(backfillTimeList.get(0)));
workflowInstance.setStartTime(new Date());
workflowInstance.setRestartTime(workflowInstance.getStartTime());
workflowInstance.setRunTimes(1);
workflowInstance.setName(String.join("-", workflowDefinition.getName(), DateUtils.getCurrentTimeStamp()));
workflowInstance.setTaskDependType(backfillTriggerRequest.getTaskDependType());
workflowInstance.setFailureStrategy(backfillTriggerRequest.getFailureStrategy());
workflowInstance
.setWarningType(ObjectUtils.defaultIfNull(backfillTriggerRequest.getWarningType(), WarningType.NONE));
workflowInstance.setWarningGroupId(backfillTriggerRequest.getWarningGroupId());
workflowInstance.setExecutorId(backfillTriggerRequest.getUserId());
workflowInstance.setExecutorName(getExecutorUser(backfillTriggerRequest.getUserId()).getUserName());
workflowInstance.setTenantCode(backfillTriggerRequest.getTenantCode());
workflowInstance.setIsSubProcess(Flag.NO);
workflowInstance.addHistoryCmd(commandType);
workflowInstance.setProcessInstancePriority(backfillTriggerRequest.getWorkflowInstancePriority());
workflowInstance
.setWorkerGroup(WorkerGroupUtils.getWorkerGroupOrDefault(backfillTriggerRequest.getWorkerGroup()));
workflowInstance.setEnvironmentCode(
EnvironmentUtils.getEnvironmentCodeOrDefault(backfillTriggerRequest.getEnvironmentCode()));
workflowInstance.setTimeout(workflowDefinition.getTimeout());
workflowInstance.setDryRun(backfillTriggerRequest.getDryRun().getCode());
workflowInstance.setTestFlag(backfillTriggerRequest.getTestFlag().getCode());
return workflowInstance;
}
@Override
protected Command constructTriggerCommand(WorkflowBackfillTriggerRequest backfillTriggerRequest,
ProcessInstance workflowInstance) {
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder()
.commandParams(backfillTriggerRequest.getStartParamList())
.startNodes(backfillTriggerRequest.getStartNodes())
.timeZone(DateUtils.getTimezone())
.backfillTimeList(backfillTriggerRequest.getBackfillTimeList())
.build();
return Command.builder()
.commandType(backfillTriggerRequest.getExecType())
.processDefinitionCode(backfillTriggerRequest.getWorkflowCode())
.processDefinitionVersion(backfillTriggerRequest.getWorkflowVersion())
.processInstanceId(workflowInstance.getId())
.processInstancePriority(workflowInstance.getProcessInstancePriority())
.commandParam(JSONUtils.toJsonString(backfillWorkflowCommandParam))
.build();
}
@Override
protected WorkflowBackfillTriggerResponse onTriggerSuccess(ProcessInstance workflowInstance) {
return WorkflowBackfillTriggerResponse.success(workflowInstance.getId());
}
}

111
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowManualTrigger.java

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerResponse;
import org.apache.commons.lang3.ObjectUtils;
import java.util.Date;
import org.springframework.stereotype.Component;
/**
* Manual trigger of the workflow, used to trigger the workflow and generate the workflow instance in the manual way.
*/
@Component
public class WorkflowManualTrigger
extends
AbstractWorkflowTrigger<WorkflowManualTriggerRequest, WorkflowManualTriggerResponse> {
@Override
protected ProcessInstance constructWorkflowInstance(final WorkflowManualTriggerRequest workflowManualTriggerRequest) {
final CommandType commandType = CommandType.START_PROCESS;
final Long workflowCode = workflowManualTriggerRequest.getWorkflowDefinitionCode();
final Integer workflowVersion = workflowManualTriggerRequest.getWorkflowDefinitionVersion();
final ProcessDefinition workflowDefinition = getProcessDefinition(workflowCode, workflowVersion);
final ProcessInstance workflowInstance = new ProcessInstance();
workflowInstance.setProcessDefinitionCode(workflowDefinition.getCode());
workflowInstance.setProcessDefinitionVersion(workflowDefinition.getVersion());
workflowInstance.setProjectCode(workflowDefinition.getProjectCode());
workflowInstance.setCommandType(commandType);
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.SUBMITTED_SUCCESS, commandType.name());
workflowInstance.setRecovery(Flag.NO);
workflowInstance.setStartTime(new Date());
workflowInstance.setRestartTime(workflowInstance.getStartTime());
workflowInstance.setRunTimes(1);
workflowInstance.setName(String.join("-", workflowDefinition.getName(), DateUtils.getCurrentTimeStamp()));
workflowInstance.setTaskDependType(workflowManualTriggerRequest.getTaskDependType());
workflowInstance.setFailureStrategy(workflowManualTriggerRequest.getFailureStrategy());
workflowInstance.setWarningType(
ObjectUtils.defaultIfNull(workflowManualTriggerRequest.getWarningType(), WarningType.NONE));
workflowInstance.setWarningGroupId(workflowManualTriggerRequest.getWarningGroupId());
workflowInstance.setExecutorId(workflowManualTriggerRequest.getUserId());
workflowInstance.setExecutorName(getExecutorUser(workflowManualTriggerRequest.getUserId()).getUserName());
workflowInstance.setTenantCode(workflowManualTriggerRequest.getTenantCode());
workflowInstance.setIsSubProcess(Flag.NO);
workflowInstance.addHistoryCmd(commandType);
workflowInstance.setProcessInstancePriority(workflowManualTriggerRequest.getWorkflowInstancePriority());
workflowInstance.setWorkerGroup(
WorkerGroupUtils.getWorkerGroupOrDefault(workflowManualTriggerRequest.getWorkerGroup()));
workflowInstance.setEnvironmentCode(
EnvironmentUtils.getEnvironmentCodeOrDefault(workflowManualTriggerRequest.getEnvironmentCode()));
workflowInstance.setTimeout(workflowDefinition.getTimeout());
workflowInstance.setDryRun(workflowManualTriggerRequest.getDryRun().getCode());
workflowInstance.setTestFlag(workflowManualTriggerRequest.getTestFlag().getCode());
return workflowInstance;
}
@Override
protected Command constructTriggerCommand(final WorkflowManualTriggerRequest workflowManualTriggerRequest,
final ProcessInstance workflowInstance) {
final RunWorkflowCommandParam runWorkflowCommandParam = RunWorkflowCommandParam.builder()
.commandParams(workflowManualTriggerRequest.getStartParamList())
.startNodes(workflowManualTriggerRequest.getStartNodes())
.timeZone(DateUtils.getTimezone())
.build();
return Command.builder()
.commandType(CommandType.START_PROCESS)
.processDefinitionCode(workflowManualTriggerRequest.getWorkflowDefinitionCode())
.processDefinitionVersion(workflowManualTriggerRequest.getWorkflowDefinitionVersion())
.processInstanceId(workflowInstance.getId())
.processInstancePriority(workflowInstance.getProcessInstancePriority())
.commandParam(JSONUtils.toJsonString(runWorkflowCommandParam))
.build();
}
@Override
protected WorkflowManualTriggerResponse onTriggerSuccess(ProcessInstance workflowInstance) {
return WorkflowManualTriggerResponse.success(workflowInstance.getId());
}
}

106
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/workflow/trigger/WorkflowScheduleTrigger.java

@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.workflow.trigger;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.master.command.ScheduleWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerResponse;
import org.apache.commons.lang3.ObjectUtils;
import java.util.Date;
import org.springframework.stereotype.Component;
@Component
public class WorkflowScheduleTrigger
extends
AbstractWorkflowTrigger<WorkflowScheduleTriggerRequest, WorkflowScheduleTriggerResponse> {
@Override
protected ProcessInstance constructWorkflowInstance(WorkflowScheduleTriggerRequest scheduleTriggerRequest) {
final CommandType commandType = CommandType.SCHEDULER;
final Long workflowCode = scheduleTriggerRequest.getWorkflowCode();
final Integer workflowVersion = scheduleTriggerRequest.getWorkflowVersion();
final ProcessDefinition workflowDefinition = getProcessDefinition(workflowCode, workflowVersion);
final ProcessInstance workflowInstance = new ProcessInstance();
workflowInstance.setProcessDefinitionCode(workflowDefinition.getCode());
workflowInstance.setProcessDefinitionVersion(workflowDefinition.getVersion());
workflowInstance.setProjectCode(workflowDefinition.getProjectCode());
workflowInstance.setCommandType(commandType);
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.SUBMITTED_SUCCESS, commandType.name());
workflowInstance.setRecovery(Flag.NO);
workflowInstance.setScheduleTime(scheduleTriggerRequest.getScheduleTIme());
workflowInstance.setStartTime(new Date());
workflowInstance.setRestartTime(workflowInstance.getStartTime());
workflowInstance.setRunTimes(1);
workflowInstance.setName(String.join("-", workflowDefinition.getName(), DateUtils.getCurrentTimeStamp()));
workflowInstance.setTaskDependType(scheduleTriggerRequest.getTaskDependType());
workflowInstance.setFailureStrategy(scheduleTriggerRequest.getFailureStrategy());
workflowInstance
.setWarningType(ObjectUtils.defaultIfNull(scheduleTriggerRequest.getWarningType(), WarningType.NONE));
workflowInstance.setWarningGroupId(scheduleTriggerRequest.getWarningGroupId());
workflowInstance.setExecutorId(scheduleTriggerRequest.getUserId());
workflowInstance.setExecutorName(getExecutorUser(scheduleTriggerRequest.getUserId()).getUserName());
workflowInstance.setTenantCode(scheduleTriggerRequest.getTenantCode());
workflowInstance.setIsSubProcess(Flag.NO);
workflowInstance.addHistoryCmd(commandType);
workflowInstance.setProcessInstancePriority(scheduleTriggerRequest.getWorkflowInstancePriority());
workflowInstance
.setWorkerGroup(WorkerGroupUtils.getWorkerGroupOrDefault(scheduleTriggerRequest.getWorkerGroup()));
workflowInstance.setEnvironmentCode(
EnvironmentUtils.getEnvironmentCodeOrDefault(scheduleTriggerRequest.getEnvironmentCode()));
workflowInstance.setTimeout(workflowDefinition.getTimeout());
workflowInstance.setDryRun(scheduleTriggerRequest.getDryRun().getCode());
workflowInstance.setTestFlag(scheduleTriggerRequest.getTestFlag().getCode());
return workflowInstance;
}
@Override
protected Command constructTriggerCommand(final WorkflowScheduleTriggerRequest scheduleTriggerRequest,
final ProcessInstance workflowInstance) {
final ScheduleWorkflowCommandParam scheduleWorkflowCommandParam = ScheduleWorkflowCommandParam.builder()
.timeZone(scheduleTriggerRequest.getTimezoneId())
.build();
return Command.builder()
.commandType(CommandType.SCHEDULER)
.processDefinitionCode(scheduleTriggerRequest.getWorkflowCode())
.processDefinitionVersion(scheduleTriggerRequest.getWorkflowVersion())
.processInstanceId(workflowInstance.getId())
.processInstancePriority(workflowInstance.getProcessInstancePriority())
.commandParam(JSONUtils.toJsonString(scheduleWorkflowCommandParam))
.build();
}
@Override
protected WorkflowScheduleTriggerResponse onTriggerSuccess(ProcessInstance workflowInstance) {
return WorkflowScheduleTriggerResponse.success(workflowInstance.getId());
}
}

14
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutionEventListenerImpl.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.rpc; package org.apache.dolphinscheduler.server.master.rpc;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent;
@ -73,7 +73,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// instance state. // instance state.
// The logic task doesn't need to send ack // The logic task doesn't need to send ack
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionDispatchEvent.getTaskInstanceHost()) .withHost(taskExecutionDispatchEvent.getTaskInstanceHost())
.handleTaskInstanceDispatchedEventAck( .handleTaskInstanceDispatchedEventAck(
@ -99,7 +99,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// instance state. // instance state.
// The logic task doesn't need to send ack // The logic task doesn't need to send ack
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionRunningEvent.getTaskInstanceHost()) .withHost(taskExecutionRunningEvent.getTaskInstanceHost())
.handleTaskInstanceExecutionRunningEventAck( .handleTaskInstanceExecutionRunningEventAck(
@ -122,7 +122,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// So once the master failover and we take over the task instance success, then we should fetch the latest task // So once the master failover and we take over the task instance success, then we should fetch the latest task
// instance state. // instance state.
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionSuccessEvent.getTaskInstanceHost()) .withHost(taskExecutionSuccessEvent.getTaskInstanceHost())
.handleTaskExecutionSuccessEventAck( .handleTaskExecutionSuccessEventAck(
@ -143,7 +143,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// So once the master failover and we take over the task instance success, then we should fetch the latest task // So once the master failover and we take over the task instance success, then we should fetch the latest task
// instance state. // instance state.
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionFailedEvent.getTaskInstanceHost()) .withHost(taskExecutionFailedEvent.getTaskInstanceHost())
.handleTaskExecutionFailedEventAck( .handleTaskExecutionFailedEventAck(
@ -164,7 +164,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// So once the master failover and we take over the task instance success, then we should fetch the latest task // So once the master failover and we take over the task instance success, then we should fetch the latest task
// instance state. // instance state.
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionKilledEvent.getTaskInstanceHost()) .withHost(taskExecutionKilledEvent.getTaskInstanceHost())
.handleTaskExecutionKilledEventAck( .handleTaskExecutionKilledEventAck(
@ -182,7 +182,7 @@ public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListen
// So once the master failover and we take over the task instance success, then we should fetch the latest task // So once the master failover and we take over the task instance success, then we should fetch the latest task
// instance state. // instance state.
if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskInstanceExecutionEventAckListener.class) .withService(ITaskInstanceExecutionEventAckListener.class)
.withHost(taskExecutionPausedEvent.getTaskInstanceHost()) .withHost(taskExecutionPausedEvent.getTaskInstanceHost())
.handleTaskExecutionPausedEventAck( .handleTaskExecutionPausedEventAck(

126
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/WorkflowControlClient.java

@ -0,0 +1,126 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.rpc;
import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerResponse;
import org.apache.dolphinscheduler.server.master.engine.WorkflowCacheRepository;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowBackfillTrigger;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowManualTrigger;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowScheduleTrigger;
import org.apache.commons.lang3.exception.ExceptionUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
@Slf4j
@Service
public class WorkflowControlClient implements IWorkflowControlClient {
@Autowired
private WorkflowManualTrigger workflowManualTrigger;
@Autowired
private WorkflowBackfillTrigger workflowBackfillTrigger;
@Autowired
private WorkflowScheduleTrigger workflowScheduleTrigger;
@Autowired
private WorkflowCacheRepository workflowRepository;
@Override
public WorkflowManualTriggerResponse manualTriggerWorkflow(final WorkflowManualTriggerRequest manualTriggerRequest) {
try {
return workflowManualTrigger.triggerWorkflow(manualTriggerRequest);
} catch (Exception ex) {
log.error("Handle workflowTriggerRequest: {} failed", manualTriggerRequest, ex);
return WorkflowManualTriggerResponse.fail("Trigger workflow failed: " + ExceptionUtils.getMessage(ex));
}
}
@Override
public WorkflowBackfillTriggerResponse backfillTriggerWorkflow(final WorkflowBackfillTriggerRequest backfillTriggerRequest) {
try {
return workflowBackfillTrigger.triggerWorkflow(backfillTriggerRequest);
} catch (Exception ex) {
log.error("Handle workflowBackfillTriggerRequest: {} failed", backfillTriggerRequest, ex);
return WorkflowBackfillTriggerResponse.fail("Backfill workflow failed: " + ExceptionUtils.getMessage(ex));
}
}
@Override
public WorkflowScheduleTriggerResponse scheduleTriggerWorkflow(WorkflowScheduleTriggerRequest workflowScheduleTriggerRequest) {
try {
return workflowScheduleTrigger.triggerWorkflow(workflowScheduleTriggerRequest);
} catch (Exception ex) {
log.error("Handle workflowScheduleTriggerRequest: {} failed", workflowScheduleTriggerRequest, ex);
return WorkflowScheduleTriggerResponse
.fail("Schedule trigger workflow failed: " + ExceptionUtils.getMessage(ex));
}
}
@Override
public WorkflowInstancePauseResponse pauseWorkflowInstance(final WorkflowInstancePauseRequest workflowInstancePauseRequest) {
try {
final Integer workflowInstanceId = workflowInstancePauseRequest.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflow = workflowRepository.get(workflowInstanceId);
if (workflow == null) {
return WorkflowInstancePauseResponse.fail(
"Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId);
}
workflow.pause();
return WorkflowInstancePauseResponse.success();
} catch (Exception ex) {
log.error("Handle workflowInstancePauseRequest: {} failed", workflowInstancePauseRequest, ex);
return WorkflowInstancePauseResponse.fail(
"Pause workflow instance failed: " + ExceptionUtils.getMessage(ex));
}
}
@Override
public WorkflowInstanceStopResponse stopWorkflowInstance(final WorkflowInstanceStopRequest workflowInstanceStopRequest) {
try {
final Integer workflowInstanceId = workflowInstanceStopRequest.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflow = workflowRepository.get(workflowInstanceId);
if (workflow == null) {
return WorkflowInstanceStopResponse
.fail("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId);
}
workflow.stop();
return WorkflowInstanceStopResponse.success();
} catch (Exception ex) {
log.error("Handle workflowInstanceStopRequest: {} failed", workflowInstanceStopRequest, ex);
return WorkflowInstanceStopResponse.fail(
"Stop workflow instance failed:" + ExceptionUtils.getMessage(ex));
}
}
}

81
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/WorkflowInstanceControllerImpl.java

@ -1,81 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.rpc;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.server.master.engine.WorkflowCacheRepository;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowPauseLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowStopLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.commons.lang3.exception.ExceptionUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
@Slf4j
@Service
public class WorkflowInstanceControllerImpl implements IWorkflowInstanceController {
@Autowired
private WorkflowCacheRepository workflowCacheRepository;
@Override
public WorkflowInstancePauseResponse pauseWorkflowInstance(final WorkflowInstancePauseRequest workflowInstancePauseRequest) {
try {
final Integer workflowInstanceId = workflowInstancePauseRequest.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflowExecutionRunnable =
workflowCacheRepository.get(workflowInstanceId);
if (workflowExecutionRunnable == null) {
return WorkflowInstancePauseResponse
.fail("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId);
}
workflowExecutionRunnable.getWorkflowEventBus()
.publish(WorkflowPauseLifecycleEvent.of(workflowExecutionRunnable));
return WorkflowInstancePauseResponse.success();
} catch (Exception ex) {
log.error("Handle workflowInstancePauseRequest: {} failed", workflowInstancePauseRequest, ex);
return WorkflowInstancePauseResponse.fail(ExceptionUtils.getMessage(ex));
}
}
@Override
public WorkflowInstanceStopResponse stopWorkflowInstance(final WorkflowInstanceStopRequest workflowInstanceStopRequest) {
try {
final Integer workflowInstanceId = workflowInstanceStopRequest.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflowExecutionRunnable =
workflowCacheRepository.get(workflowInstanceId);
if (workflowExecutionRunnable == null) {
return WorkflowInstanceStopResponse
.fail("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId);
}
workflowExecutionRunnable.getWorkflowEventBus()
.publish(WorkflowStopLifecycleEvent.of(workflowExecutionRunnable));
return WorkflowInstanceStopResponse.success();
} catch (Exception ex) {
log.error("Handle workflowInstanceStopRequest: {} failed", workflowInstanceStopRequest, ex);
return WorkflowInstanceStopResponse.fail(ExceptionUtils.getMessage(ex));
}
}
}

5
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteContext.java

@ -77,11 +77,6 @@ public class WorkflowExecuteContext implements IWorkflowExecuteContext {
return this; return this;
} }
public WorkflowExecuteContextBuilder withWorkflowInstanceLifecycleListeners(List<IWorkflowLifecycleListener> workflowLifecycleListeners) {
this.workflowInstanceLifecycleListeners = workflowLifecycleListeners;
return this;
}
public WorkflowExecuteContext build() { public WorkflowExecuteContext build() {
return new WorkflowExecuteContext( return new WorkflowExecuteContext(
command, command,

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java

@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.master.runner.dispatcher;
import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.extract.base.utils.Host;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchRequest; import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchRequest;
@ -51,7 +51,7 @@ public class WorkerTaskDispatcher extends BaseTaskDispatcher {
final String taskName = taskExecutionContext.getTaskName(); final String taskName = taskExecutionContext.getTaskName();
final String workerAddress = taskExecutionContext.getHost(); final String workerAddress = taskExecutionContext.getHost();
try { try {
final TaskInstanceDispatchResponse taskInstanceDispatchResponse = SingletonJdkDynamicRpcClientProxyFactory final TaskInstanceDispatchResponse taskInstanceDispatchResponse = Clients
.withService(ITaskInstanceOperator.class) .withService(ITaskInstanceOperator.class)
.withHost(workerAddress) .withHost(workerAddress)
.dispatchTask(new TaskInstanceDispatchRequest(ITaskExecutionRunnable.getTaskExecutionContext())); .dispatchTask(new TaskInstanceDispatchRequest(ITaskExecutionRunnable.getTaskExecutionContext()));

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteDispatchEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -31,7 +31,7 @@ public class LogicTaskInstanceExecuteDispatchEventSender
@Override @Override
public void sendMessage(TaskExecutionDispatchEvent taskExecutionDispatchEvent) { public void sendMessage(TaskExecutionDispatchEvent taskExecutionDispatchEvent) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost()) .withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost())
.onTaskInstanceDispatched(taskExecutionDispatchEvent); .onTaskInstanceDispatched(taskExecutionDispatchEvent);

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -33,7 +33,7 @@ public class LogicTaskInstanceExecuteRunningEventSender
@Override @Override
public void sendMessage(TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) { public void sendMessage(TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost()) .withHost(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost())
.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent); .onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFailedEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -31,7 +31,7 @@ public class LogicTaskInstanceExecutionFailedEventSender
@Override @Override
public void sendMessage(final TaskExecutionFailedEvent message) { public void sendMessage(final TaskExecutionFailedEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionFailed(message); .onTaskInstanceExecutionFailed(message);

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionKilledEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -31,7 +31,7 @@ public class LogicTaskInstanceExecutionKilledEventSender
@Override @Override
public void sendMessage(final TaskExecutionKilledEvent message) { public void sendMessage(final TaskExecutionKilledEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionKilled(message); .onTaskInstanceExecutionKilled(message);

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionPausedEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -31,7 +31,7 @@ public class LogicTaskInstanceExecutionPausedEventSender
@Override @Override
public void sendMessage(final TaskExecutionPausedEvent taskExecutionPausedEvent) { public void sendMessage(final TaskExecutionPausedEvent taskExecutionPausedEvent) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(taskExecutionPausedEvent.getWorkflowInstanceHost()) .withHost(taskExecutionPausedEvent.getWorkflowInstanceHost())
.onTaskInstanceExecutionPaused(taskExecutionPausedEvent); .onTaskInstanceExecutionPaused(taskExecutionPausedEvent);

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionSuccessEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.master.runner.message; package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
@ -31,7 +31,7 @@ public class LogicTaskInstanceExecutionSuccessEventSender
@Override @Override
public void sendMessage(final TaskExecutionSuccessEvent message) { public void sendMessage(final TaskExecutionSuccessEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionSuccess(message); .onTaskInstanceExecutionSuccess(message);

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

@ -30,10 +30,10 @@ import org.apache.dolphinscheduler.dao.mapper.CommandMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.DynamicInputParameter; import org.apache.dolphinscheduler.plugin.task.api.model.DynamicInputParameter;
import org.apache.dolphinscheduler.plugin.task.api.model.Property; import org.apache.dolphinscheduler.plugin.task.api.model.Property;
@ -283,8 +283,8 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
private void doKillRunningSubWorkflowInstances(List<ProcessInstance> runningSubProcessInstanceList) throws MasterTaskExecuteException { private void doKillRunningSubWorkflowInstances(List<ProcessInstance> runningSubProcessInstanceList) throws MasterTaskExecuteException {
for (ProcessInstance subProcessInstance : runningSubProcessInstanceList) { for (ProcessInstance subProcessInstance : runningSubProcessInstanceList) {
try { try {
WorkflowInstanceStopResponse workflowInstanceStopResponse = SingletonJdkDynamicRpcClientProxyFactory WorkflowInstanceStopResponse workflowInstanceStopResponse = Clients
.withService(IWorkflowInstanceController.class) .withService(IWorkflowControlClient.class)
.withHost(subProcessInstance.getHost()) .withHost(subProcessInstance.getHost())
.stopWorkflowInstance(new WorkflowInstanceStopRequest(subProcessInstance.getId())); .stopWorkflowInstance(new WorkflowInstanceStopRequest(subProcessInstance.getId()));
if (workflowInstanceStopResponse.isSuccess()) { if (workflowInstanceStopResponse.isSuccess()) {

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

@ -25,14 +25,14 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.command.ICommandParam; import org.apache.dolphinscheduler.extract.master.command.ICommandParam;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters; import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
@ -99,8 +99,8 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameter
taskExecutionContext.getTaskInstanceId()); taskExecutionContext.getTaskInstanceId());
try { try {
WorkflowInstancePauseResponse pauseResponse = SingletonJdkDynamicRpcClientProxyFactory WorkflowInstancePauseResponse pauseResponse = Clients
.withService(IWorkflowInstanceController.class) .withService(IWorkflowControlClient.class)
.withHost(subProcessInstance.getHost()) .withHost(subProcessInstance.getHost())
.pauseWorkflowInstance(new WorkflowInstancePauseRequest(subProcessInstance.getId())); .pauseWorkflowInstance(new WorkflowInstancePauseRequest(subProcessInstance.getId()));
if (pauseResponse.isSuccess()) { if (pauseResponse.isSuccess()) {
@ -128,8 +128,8 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameter
return; return;
} }
try { try {
WorkflowInstanceStopResponse stopResponse = SingletonJdkDynamicRpcClientProxyFactory WorkflowInstanceStopResponse stopResponse = Clients
.withService(IWorkflowInstanceController.class) .withService(IWorkflowControlClient.class)
.withHost(subProcessInstance.getHost()) .withHost(subProcessInstance.getHost())
.stopWorkflowInstance(new WorkflowInstanceStopRequest(subProcessInstance.getId())); .stopWorkflowInstance(new WorkflowInstanceStopRequest(subProcessInstance.getId()));
if (stopResponse.isSuccess()) { if (stopResponse.isSuccess()) {

14
dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/it/Repository.java

@ -53,6 +53,10 @@ public class Repository {
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
public ProcessInstance queryWorkflowInstance(final Integer workflowInstanceId) {
return workflowInstanceDao.queryById(workflowInstanceId);
}
/** /**
* Return the list of task instances for a given workflow definition in ascending order of their IDs. * Return the list of task instances for a given workflow definition in ascending order of their IDs.
*/ */
@ -65,4 +69,14 @@ public class Repository {
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
/**
* Return the list of task instances for a given workflow definition in ascending order of their IDs.
*/
public List<TaskInstance> queryTaskInstance(final Integer workflowInstanceId) {
return taskInstanceDao.queryByWorkflowInstanceId(workflowInstanceId)
.stream()
.sorted(Comparator.comparingInt(TaskInstance::getId))
.collect(Collectors.toList());
}
} }

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

@ -17,29 +17,27 @@
package org.apache.dolphinscheduler.server.master.it; package org.apache.dolphinscheduler.server.master.it;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.repository.CommandDao; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController;
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerResponse;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstancePauseResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerResponse;
import org.apache.dolphinscheduler.scheduler.api.SchedulerApi; import org.apache.dolphinscheduler.scheduler.api.SchedulerApi;
import java.util.Date;
import lombok.AllArgsConstructor; import lombok.AllArgsConstructor;
import lombok.Builder; import lombok.Builder;
import lombok.Data; import lombok.Data;
import org.assertj.core.api.Assertions;
import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component; import org.springframework.stereotype.Component;
@ -47,28 +45,42 @@ import org.springframework.stereotype.Component;
public class WorkflowOperator { public class WorkflowOperator {
@Autowired @Autowired
private CommandDao commandDao; private IWorkflowControlClient workflowInstanceController;
@Autowired
private IWorkflowInstanceController workflowInstanceController;
@Autowired @Autowired
private SchedulerApi schedulerApi; private SchedulerApi schedulerApi;
public Integer manualTriggerWorkflow(final WorkflowTriggerDTO workflowTriggerDTO) {
final WorkflowManualTriggerRequest workflowManualTriggerRequest = WorkflowManualTriggerRequest.builder()
.userId(workflowTriggerDTO.workflowDefinition.getUserId())
.workflowDefinitionCode(workflowTriggerDTO.workflowDefinition.getCode())
.workflowDefinitionVersion(workflowTriggerDTO.workflowDefinition.getVersion())
.startNodes(workflowTriggerDTO.getRunWorkflowCommandParam().getStartNodes())
.startParamList(workflowTriggerDTO.getRunWorkflowCommandParam().getCommandParams())
.build();
final WorkflowManualTriggerResponse manualTriggerWorkflowResponse =
workflowInstanceController.manualTriggerWorkflow(workflowManualTriggerRequest);
Assertions.assertThat(manualTriggerWorkflowResponse.isSuccess()).isTrue();
return manualTriggerWorkflowResponse.getWorkflowInstanceId();
}
public void backfillWorkflow(final WorkflowBackfillDTO workflowBackfillDTO) { public void backfillWorkflow(final WorkflowBackfillDTO workflowBackfillDTO) {
final ProcessDefinition workflowDefinition = workflowBackfillDTO.getWorkflow(); final ProcessDefinition workflowDefinition = workflowBackfillDTO.getWorkflow();
final Command command = Command.builder()
.commandType(CommandType.COMPLEMENT_DATA) final WorkflowBackfillTriggerRequest backfillTriggerRequest = WorkflowBackfillTriggerRequest.builder()
.processDefinitionCode(workflowDefinition.getCode()) .userId(workflowDefinition.getUserId())
.processDefinitionVersion(workflowDefinition.getVersion()) .workflowCode(workflowDefinition.getCode())
.executorId(workflowDefinition.getUserId()) .workflowVersion(workflowDefinition.getVersion())
.scheduleTime(DateUtils.stringToDate( .startNodes(workflowBackfillDTO.getBackfillWorkflowCommandParam().getStartNodes())
workflowBackfillDTO.getBackfillWorkflowCommandParam().getBackfillTimeList().get(0))) .startParamList(workflowBackfillDTO.getBackfillWorkflowCommandParam().getCommandParams())
.commandParam(JSONUtils.toJsonString(workflowBackfillDTO.getBackfillWorkflowCommandParam())) .backfillTimeList(workflowBackfillDTO.getBackfillWorkflowCommandParam().getBackfillTimeList())
.startTime(new Date())
.updateTime(new Date())
.build(); .build();
commandDao.insert(command); final WorkflowBackfillTriggerResponse backfillTriggerResponse =
workflowInstanceController.backfillTriggerWorkflow(backfillTriggerRequest);
Assertions.assertThat(backfillTriggerResponse.isSuccess()).isTrue();
} }
public void schedulingWorkflow(final WorkflowSchedulingDTO workflowSchedulingDTO) { public void schedulingWorkflow(final WorkflowSchedulingDTO workflowSchedulingDTO) {
@ -77,21 +89,6 @@ public class WorkflowOperator {
schedulerApi.insertOrUpdateScheduleTask(project.getId(), schedule); schedulerApi.insertOrUpdateScheduleTask(project.getId(), schedule);
} }
public void triggerWorkflow(final WorkflowTriggerDTO workflowTriggerDTO) {
final ProcessDefinition workflowDefinition = workflowTriggerDTO.getWorkflowDefinition();
final RunWorkflowCommandParam runWorkflowCommandParam = workflowTriggerDTO.getRunWorkflowCommandParam();
final Command command = Command.builder()
.commandType(CommandType.START_PROCESS)
.processDefinitionCode(workflowDefinition.getCode())
.processDefinitionVersion(workflowDefinition.getVersion())
.executorId(workflowDefinition.getUserId())
.commandParam(JSONUtils.toJsonString(runWorkflowCommandParam))
.startTime(new Date())
.updateTime(new Date())
.build();
commandDao.insert(command);
}
public WorkflowInstancePauseResponse pauseWorkflowInstance(Integer workflowInstanceId) { public WorkflowInstancePauseResponse pauseWorkflowInstance(Integer workflowInstanceId) {
final WorkflowInstancePauseRequest workflowInstancePauseRequest = final WorkflowInstancePauseRequest workflowInstancePauseRequest =
new WorkflowInstancePauseRequest(workflowInstanceId); new WorkflowInstancePauseRequest(workflowInstanceId);

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

@ -21,7 +21,6 @@ import static com.google.common.truth.Truth.assertThat;
import static org.awaitility.Awaitility.await; import static org.awaitility.Awaitility.await;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@ -67,30 +66,37 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.pollInterval(Duration.ofMillis(100)) .pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> {
Assertions
// Make sure the task instance is running, otherwise will pause the submitted task instance .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
// In fact, this case might pause the first three tasks if the first three tasks are not dispatched. .satisfies(workflowInstance -> {
ThreadUtils.sleep(1_000); assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
});
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess()); assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess());
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfiesExactly(workflowInstance -> { .satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.SUCCESS); assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.SUCCESS);
}); });
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> { .satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("A"); assertThat(taskInstance.getName()).isEqualTo("A");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS); assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.SUCCESS);
@ -111,30 +117,38 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.pollInterval(Duration.ofMillis(100)) .pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> {
Assertions
// Make sure the task instance is running, otherwise will pause the submitted task instance .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
// In fact, this case might pause the first three tasks if the first three tasks are not dispatched. .satisfies(workflowInstance -> {
ThreadUtils.sleep(1_000); assertThat(workflowInstance.getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(
repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
});
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess()); assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess());
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfiesExactly(workflowInstance -> { .satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.FAILURE); assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.FAILURE);
}); });
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(taskInstance -> { .satisfiesExactly(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("A"); assertThat(taskInstance.getName()).isEqualTo("A");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE); assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.FAILURE);
@ -154,30 +168,45 @@ public class WorkflowInstancePauseIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.pollInterval(Duration.ofMillis(100)) .pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> {
Assertions.assertThat(repository.queryWorkflowInstance(workflowInstanceId))
// Make sure the task instance is running, otherwise will pause the submitted task instance .satisfies(workflowInstance -> {
// In fact, this case might pause the first three tasks if the first three tasks are not dispatched. assertThat(workflowInstance.getState())
ThreadUtils.sleep(1_000); .isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
});
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.anySatisfy(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("A1");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
})
.anySatisfy(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("B1");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
})
.anySatisfy(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("C1");
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
});
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess()); assertThat(workflowOperator.pauseWorkflowInstance(workflowInstanceId).isSuccess());
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfiesExactly(workflowInstance -> { .satisfies(workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.PAUSE); assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.PAUSE);
}); });
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.hasSize(6) .hasSize(6)
.anySatisfy(taskInstance -> { .anySatisfy(taskInstance -> {
assertThat(taskInstance.getName()).isEqualTo("A1"); assertThat(taskInstance.getName()).isEqualTo("A1");

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

@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam;
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest; import org.apache.dolphinscheduler.server.master.AbstractMasterIntegrationTest;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
@ -68,14 +68,24 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.pollInterval(Duration.ofMillis(100)) .pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId).getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(
taskInstance -> {
assertThat(taskInstance.getState())
.isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
});
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
Assertions Assertions
.assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId)) .assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId))
.matches(WorkflowInstanceStopResponse::isSuccess); .matches(WorkflowInstanceStopResponse::isSuccess);
@ -84,13 +94,10 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId).getState())
.satisfiesExactly( .isEqualTo(WorkflowExecutionStatus.STOP);
workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
});
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly( .satisfiesExactly(
taskInstance -> { taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL); assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL);
@ -110,27 +117,36 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.pollInterval(Duration.ofMillis(100)) .pollInterval(Duration.ofMillis(100))
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> {
Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId).getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION);
Assertions
.assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly(
taskInstance -> {
assertThat(taskInstance.getState())
.isEqualTo(TaskExecutionStatus.RUNNING_EXECUTION);
});
});
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess()); assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess());
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId))
.satisfiesExactly( .satisfies(
workflowInstance -> { workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP); assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
}); });
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.satisfiesExactly( .satisfiesExactly(
taskInstance -> { taskInstance -> {
assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL); assertThat(taskInstance.getState()).isEqualTo(TaskExecutionStatus.KILL);
@ -150,29 +166,29 @@ public class WorkflowInstanceStopIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); final Integer workflowInstanceId = workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> Assertions.assertThat(repository.queryWorkflowInstance(workflow)).hasSize(1)); .untilAsserted(() -> Assertions
.assertThat(repository.queryWorkflowInstance(workflowInstanceId).getState())
.isEqualTo(WorkflowExecutionStatus.RUNNING_EXECUTION)
);
// make sure the task has been dispatched to the executor // make sure the task has been dispatched to the executor
ThreadUtils.sleep(2_000); ThreadUtils.sleep(2_000);
final Integer workflowInstanceId = repository.queryWorkflowInstance(workflow).get(0).getId();
assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess()); assertThat(workflowOperator.stopWorkflowInstance(workflowInstanceId).isSuccess());
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
.untilAsserted(() -> { .untilAsserted(() -> {
Assertions Assertions
.assertThat(repository.queryWorkflowInstance(workflow)) .assertThat(repository.queryWorkflowInstance(workflowInstanceId).getState())
.satisfiesExactly( .isEqualTo(WorkflowExecutionStatus.STOP);
workflowInstance -> {
assertThat(workflowInstance.getState()).isEqualTo(WorkflowExecutionStatus.STOP);
});
Assertions Assertions
.assertThat(repository.queryTaskInstance(workflow)) .assertThat(repository.queryTaskInstance(workflowInstanceId))
.hasSize(3) .hasSize(3)
.anySatisfy( .anySatisfy(
taskInstance -> { taskInstance -> {

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

@ -78,7 +78,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -109,7 +109,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -154,7 +154,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(runWorkflowCommandParam) .runWorkflowCommandParam(runWorkflowCommandParam)
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -190,7 +190,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -221,7 +221,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(3)) .atMost(Duration.ofMinutes(3))
@ -274,7 +274,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -313,7 +313,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -344,7 +344,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -382,7 +382,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -421,7 +421,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(new RunWorkflowCommandParam()) .runWorkflowCommandParam(new RunWorkflowCommandParam())
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))
@ -491,7 +491,7 @@ public class WorkflowStartIT extends AbstractMasterIntegrationTest {
.workflowDefinition(workflow) .workflowDefinition(workflow)
.runWorkflowCommandParam(runWorkflowCommandParam) .runWorkflowCommandParam(runWorkflowCommandParam)
.build(); .build();
workflowOperator.triggerWorkflow(workflowTriggerDTO); workflowOperator.manualTriggerWorkflow(workflowTriggerDTO);
await() await()
.atMost(Duration.ofMinutes(1)) .atMost(Duration.ofMinutes(1))

7
dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/RpcBenchMarkTest.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.microbench.rpc; package org.apache.dolphinscheduler.microbench.rpc;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.base.config.NettyServerConfig; import org.apache.dolphinscheduler.extract.base.config.NettyServerConfig;
import org.apache.dolphinscheduler.extract.base.server.SpringServerMethodInvokerDiscovery; import org.apache.dolphinscheduler.extract.base.server.SpringServerMethodInvokerDiscovery;
import org.apache.dolphinscheduler.microbench.base.AbstractBaseBenchmark; import org.apache.dolphinscheduler.microbench.base.AbstractBaseBenchmark;
@ -56,8 +56,9 @@ public class RpcBenchMarkTest extends AbstractBaseBenchmark {
springServerMethodInvokerDiscovery = new SpringServerMethodInvokerDiscovery(nettyServerConfig); springServerMethodInvokerDiscovery = new SpringServerMethodInvokerDiscovery(nettyServerConfig);
springServerMethodInvokerDiscovery.postProcessAfterInitialization(new IServiceImpl(), "iServiceImpl"); springServerMethodInvokerDiscovery.postProcessAfterInitialization(new IServiceImpl(), "iServiceImpl");
springServerMethodInvokerDiscovery.start(); springServerMethodInvokerDiscovery.start();
iService = iService = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient("localhost:12345", IService.class); .withService(IService.class)
.withHost("localhost:12345");
} }
@Benchmark @Benchmark

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

@ -28,6 +28,8 @@ import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import java.io.IOException; import java.io.IOException;
@ -39,6 +41,7 @@ import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import lombok.NonNull; import lombok.NonNull;
@ -129,6 +132,15 @@ public class RegistryClient {
return serverList; return serverList;
} }
public Optional<Server> getRandomServer(final RegistryNodeType registryNodeType) {
final List<Server> serverList = getServerList(registryNodeType);
if (CollectionUtils.isEmpty(serverList)) {
return Optional.empty();
}
final Server server = serverList.get(RandomUtils.nextInt(0, serverList.size()));
return Optional.ofNullable(server);
}
/** /**
* Return server host:port -> value * Return server host:port -> value
*/ */

5
dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/pom.xml

@ -40,6 +40,11 @@
<artifactId>dolphinscheduler-meter</artifactId> <artifactId>dolphinscheduler-meter</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-master</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.springframework.boot</groupId> <groupId>org.springframework.boot</groupId>
<artifactId>spring-boot-starter-quartz</artifactId> <artifactId>spring-boot-starter-quartz</artifactId>

49
dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java

@ -17,14 +17,14 @@
package org.apache.dolphinscheduler.scheduler.quartz; package org.apache.dolphinscheduler.scheduler.quartz;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.ReleaseState; import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.master.command.ScheduleWorkflowCommandParam; import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowScheduleTriggerRequest;
import org.apache.dolphinscheduler.service.command.CommandService; import org.apache.dolphinscheduler.service.command.CommandService;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
@ -50,6 +50,9 @@ public class ProcessScheduleTask extends QuartzJobBean {
@Autowired @Autowired
private CommandService commandService; private CommandService commandService;
@Autowired
private IWorkflowControlClient workflowInstanceController;
@Counted(value = "ds.master.quartz.job.executed") @Counted(value = "ds.master.quartz.job.executed")
@Timed(value = "ds.master.quartz.job.execution.time", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true) @Timed(value = "ds.master.quartz.job.execution.time", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
@Override @Override
@ -85,28 +88,24 @@ public class ProcessScheduleTask extends QuartzJobBean {
return; return;
} }
Command command = new Command(); final WorkflowScheduleTriggerRequest scheduleTriggerRequest = WorkflowScheduleTriggerRequest.builder()
command.setCommandType(CommandType.SCHEDULER); .userId(schedule.getUserId())
command.setExecutorId(schedule.getUserId()); .scheduleTIme(scheduledFireTime)
command.setFailureStrategy(schedule.getFailureStrategy()); .timezoneId(schedule.getTimezoneId())
command.setProcessDefinitionCode(schedule.getProcessDefinitionCode()); .workflowCode(processDefinition.getCode())
command.setScheduleTime(scheduledFireTime); .workflowVersion(processDefinition.getVersion())
command.setStartTime(fireTime); .failureStrategy(schedule.getFailureStrategy())
command.setWarningGroupId(schedule.getWarningGroupId()); .taskDependType(TaskDependType.TASK_POST)
String workerGroup = WorkerGroupUtils.getWorkerGroupOrDefault(schedule.getWorkerGroup()); .warningType(schedule.getWarningType())
command.setWorkerGroup(workerGroup); .warningGroupId(schedule.getWarningGroupId())
command.setTenantCode(schedule.getTenantCode()); .workflowInstancePriority(schedule.getProcessInstancePriority())
command.setEnvironmentCode(schedule.getEnvironmentCode()); .workerGroup(WorkerGroupUtils.getWorkerGroupOrDefault(schedule.getWorkerGroup()))
command.setWarningType(schedule.getWarningType()); .tenantCode(schedule.getTenantCode())
command.setProcessInstancePriority(schedule.getProcessInstancePriority()); .environmentCode(schedule.getEnvironmentCode())
command.setProcessDefinitionVersion(processDefinition.getVersion()); .dryRun(Flag.NO)
.testFlag(Flag.NO)
final ScheduleWorkflowCommandParam scheduleWorkflowCommandParam = ScheduleWorkflowCommandParam.builder()
.timeZone(schedule.getTimezoneId())
.build(); .build();
command.setCommandParam(JSONUtils.toJsonString(scheduleWorkflowCommandParam)); workflowInstanceController.scheduleTriggerWorkflow(scheduleTriggerRequest);
commandService.createCommand(command);
} }
private void deleteJob(JobExecutionContext context, int projectId, int scheduleId) { private void deleteJob(JobExecutionContext context, int projectId, int scheduleId) {

12
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java

@ -105,7 +105,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.utils.DqRuleUtils; import org.apache.dolphinscheduler.dao.utils.DqRuleUtils;
import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils; import org.apache.dolphinscheduler.dao.utils.EnvironmentUtils;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.common.ILogService; import org.apache.dolphinscheduler.extract.common.ILogService;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; 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.enums.TaskExecutionStatus;
@ -261,9 +261,6 @@ public class ProcessServiceImpl implements ProcessService {
@Autowired @Autowired
private CommandService commandService; private CommandService commandService;
@Autowired
private TriggerRelationService triggerRelationService;
/** /**
* find process instance detail by id * find process instance detail by id
* *
@ -362,9 +359,10 @@ public class ProcessServiceImpl implements ProcessService {
if (StringUtils.isEmpty(taskInstance.getHost()) || StringUtils.isEmpty(taskLogPath)) { if (StringUtils.isEmpty(taskInstance.getHost()) || StringUtils.isEmpty(taskLogPath)) {
continue; continue;
} }
ILogService iLogService = Clients
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(taskInstance.getHost(), ILogService.class); .withService(ILogService.class)
iLogService.removeTaskInstanceLog(taskLogPath); .withHost(taskInstance.getHost())
.removeTaskInstanceLog(taskLogPath);
} }
} }

40
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/TriggerRelationService.java

@ -1,40 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.service.process;
import org.apache.dolphinscheduler.common.enums.ApiTriggerType;
import org.apache.dolphinscheduler.dao.entity.TriggerRelation;
import java.util.List;
import org.springframework.stereotype.Component;
/**
* Trigger relation operator to dbbecause operator command process instance
*/
@Component
public interface TriggerRelationService {
void saveTriggerToDb(ApiTriggerType type, Long triggerCode, Integer jobId);
List<TriggerRelation> queryByTypeAndJobId(ApiTriggerType apiTriggerType, int jobId);
int saveCommandTrigger(Integer commandId, Integer processInstanceId);
int saveProcessInstanceTrigger(Integer commandId, Integer processInstanceId);
}

94
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/TriggerRelationServiceImpl.java

@ -1,94 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.service.process;
import org.apache.dolphinscheduler.common.enums.ApiTriggerType;
import org.apache.dolphinscheduler.dao.entity.TriggerRelation;
import org.apache.dolphinscheduler.dao.mapper.TriggerRelationMapper;
import org.apache.commons.collections4.CollectionUtils;
import java.util.Date;
import java.util.List;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
/**
* Trigger relation operator to db
*/
@Slf4j
@Component
public class TriggerRelationServiceImpl implements TriggerRelationService {
@Autowired
private TriggerRelationMapper triggerRelationMapper;
@Override
public void saveTriggerToDb(ApiTriggerType type, Long triggerCode, Integer jobId) {
TriggerRelation triggerRelation = new TriggerRelation();
triggerRelation.setTriggerType(type.getCode());
triggerRelation.setJobId(jobId);
triggerRelation.setTriggerCode(triggerCode);
triggerRelation.setCreateTime(new Date());
triggerRelation.setUpdateTime(new Date());
triggerRelationMapper.upsert(triggerRelation);
}
@Override
public List<TriggerRelation> queryByTypeAndJobId(ApiTriggerType apiTriggerType, int jobId) {
return triggerRelationMapper.queryByTypeAndJobId(apiTriggerType.getCode(), jobId);
}
@Override
public int saveCommandTrigger(Integer commandId, Integer processInstanceId) {
List<TriggerRelation> existTriggers = queryByTypeAndJobId(ApiTriggerType.PROCESS, processInstanceId);
if (CollectionUtils.isEmpty(existTriggers)) {
return 0;
}
existTriggers.forEach(triggerRelation -> saveTriggerToDb(ApiTriggerType.COMMAND,
triggerRelation.getTriggerCode(), commandId));
int triggerRelationSize = existTriggers.size();
if (triggerRelationSize > 1) {
// Fix https://github.com/apache/dolphinscheduler/issues/15864
// This case shouldn't happen
log.error("The PROCESS TriggerRelation of command: {} is more than one", commandId);
}
return existTriggers.size();
}
@Override
public int saveProcessInstanceTrigger(Integer commandId, Integer processInstanceId) {
List<TriggerRelation> existTriggers = queryByTypeAndJobId(ApiTriggerType.COMMAND, commandId);
if (CollectionUtils.isEmpty(existTriggers)) {
return 0;
}
existTriggers.forEach(triggerRelation -> saveTriggerToDb(ApiTriggerType.PROCESS,
triggerRelation.getTriggerCode(), processInstanceId));
int triggerRelationSize = existTriggers.size();
if (triggerRelationSize > 1) {
// Fix https://github.com/apache/dolphinscheduler/issues/15864
// This case shouldn't happen
log.error("The COMMAND TriggerRelation of command: {} is more than one", commandId);
}
return existTriggers.size();
}
}

3
dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java

@ -167,9 +167,6 @@ public class ProcessServiceTest {
@Mock @Mock
TaskPluginManager taskPluginManager; TaskPluginManager taskPluginManager;
@Mock
private TriggerRelationService triggerRelationService;
@Test @Test
public void testGetUserById() { public void testGetUserById() {
User user = new User(); User user = new User();

97
dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/TriggerRelationServiceTest.java

@ -1,97 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.service.process;
import static com.google.common.truth.Truth.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.when;
import org.apache.dolphinscheduler.common.enums.ApiTriggerType;
import org.apache.dolphinscheduler.dao.entity.TriggerRelation;
import org.apache.dolphinscheduler.dao.mapper.TriggerRelationMapper;
import java.util.Date;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import com.google.common.collect.Lists;
/**
* Trigger Relation Service Test
*/
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.LENIENT)
public class TriggerRelationServiceTest {
@InjectMocks
private TriggerRelationServiceImpl triggerRelationService;
@Mock
private TriggerRelationMapper triggerRelationMapper;
@Test
public void saveTriggerToDb() {
doNothing().when(triggerRelationMapper).upsert(any());
triggerRelationService.saveTriggerToDb(ApiTriggerType.COMMAND, 1234567890L, 100);
}
@Test
public void queryByTypeAndJobId() {
doNothing().when(triggerRelationMapper).upsert(any());
when(triggerRelationMapper.queryByTypeAndJobId(ApiTriggerType.PROCESS.getCode(), 100))
.thenReturn(Lists.newArrayList(getTriggerTdoDb()));
assertThat(triggerRelationService.queryByTypeAndJobId(ApiTriggerType.PROCESS, 100)).hasSize(1);
assertThat(triggerRelationService.queryByTypeAndJobId(ApiTriggerType.PROCESS, 200)).isEmpty();
}
@Test
public void saveCommandTrigger() {
doNothing().when(triggerRelationMapper).upsert(any());
when(triggerRelationMapper.queryByTypeAndJobId(ApiTriggerType.PROCESS.getCode(), 100))
.thenReturn(Lists.newArrayList(getTriggerTdoDb()));
assertThat(triggerRelationService.saveCommandTrigger(1234567890, 100)).isAtLeast(1);
assertThat(triggerRelationService.saveCommandTrigger(1234567890, 200)).isEqualTo(0);
}
@Test
public void saveProcessInstanceTrigger() {
doNothing().when(triggerRelationMapper).upsert(any());
when(triggerRelationMapper.queryByTypeAndJobId(ApiTriggerType.COMMAND.getCode(), 100))
.thenReturn(Lists.newArrayList(getTriggerTdoDb()));
assertThat(triggerRelationService.saveProcessInstanceTrigger(100, 1234567890)).isAtLeast(1);
assertThat(triggerRelationService.saveProcessInstanceTrigger(200, 1234567890)).isEqualTo(0);
}
private TriggerRelation getTriggerTdoDb() {
TriggerRelation triggerRelation = new TriggerRelation();
triggerRelation.setTriggerType(ApiTriggerType.PROCESS.getCode());
triggerRelation.setJobId(100);
triggerRelation.setTriggerCode(1234567890L);
triggerRelation.setCreateTime(new Date());
triggerRelation.setUpdateTime(new Date());
return triggerRelation;
}
}

5
dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag-startup-param.tsx

@ -117,10 +117,9 @@ export default defineComponent({
<span class={styles.tab}> <span class={styles.tab}>
{t('project.workflow.complement_range')}: {t('project.workflow.complement_range')}:
</span> </span>
{this.commandParam && this.commandParam.complementStartDate ? ( {this.commandParam && this.commandParam.backfillTimeList ? (
<span class={styles.content}> <span class={styles.content}>
{this.commandParam.complementStartDate}- {this.commandParam.backfillTimeList.join(',')}
{this.commandParam.complementEndDate}
</span> </span>
) : ( ) : (
'-' '-'

4
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionDispatchEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent;
@ -32,7 +32,7 @@ public class TaskExecutionDispatchEventSender
@Override @Override
public void sendEvent(TaskExecutionDispatchEvent taskExecutionDispatchEvent) { public void sendEvent(TaskExecutionDispatchEvent taskExecutionDispatchEvent) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost()) .withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost())
.onTaskInstanceDispatched(taskExecutionDispatchEvent); .onTaskInstanceDispatched(taskExecutionDispatchEvent);

4
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionFailedEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent;
@ -32,7 +32,7 @@ public class TaskExecutionFailedEventSender
@Override @Override
public void sendEvent(TaskExecutionFailedEvent message) { public void sendEvent(TaskExecutionFailedEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionFailed(message); .onTaskInstanceExecutionFailed(message);

4
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionKilledEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent;
@ -32,7 +32,7 @@ public class TaskExecutionKilledEventSender
@Override @Override
public void sendEvent(TaskExecutionKilledEvent message) { public void sendEvent(TaskExecutionKilledEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionKilled(message); .onTaskInstanceExecutionKilled(message);

4
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionPausedEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent;
@ -32,7 +32,7 @@ public class TaskExecutionPausedEventSender
@Override @Override
public void sendEvent(TaskExecutionPausedEvent taskExecutionPausedEvent) { public void sendEvent(TaskExecutionPausedEvent taskExecutionPausedEvent) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(taskExecutionPausedEvent.getWorkflowInstanceHost()) .withHost(taskExecutionPausedEvent.getWorkflowInstanceHost())
.onTaskInstanceExecutionPaused(taskExecutionPausedEvent); .onTaskInstanceExecutionPaused(taskExecutionPausedEvent);

4
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionSuccessEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent;
@ -32,7 +32,7 @@ public class TaskExecutionSuccessEventSender
@Override @Override
public void sendEvent(TaskExecutionSuccessEvent message) { public void sendEvent(TaskExecutionSuccessEvent message) {
SingletonJdkDynamicRpcClientProxyFactory Clients
.withService(ITaskExecutionEventListener.class) .withService(ITaskExecutionEventListener.class)
.withHost(message.getWorkflowInstanceHost()) .withHost(message.getWorkflowInstanceHost())
.onTaskInstanceExecutionSuccess(message); .onTaskInstanceExecutionSuccess(message);

12
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.message; package org.apache.dolphinscheduler.server.worker.message;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent; import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent;
@ -33,12 +33,10 @@ public class TaskInstanceExecutionRunningEventSender
TaskInstanceExecutionEventSender<TaskExecutionRunningEvent> { TaskInstanceExecutionEventSender<TaskExecutionRunningEvent> {
@Override @Override
public void sendEvent(TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) { public void sendEvent(final TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) {
ITaskExecutionEventListener iTaskExecutionEventListener = Clients.withService(ITaskExecutionEventListener.class)
SingletonJdkDynamicRpcClientProxyFactory .withHost(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost())
.getProxyClient(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost(), .onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);
ITaskExecutionEventListener.class);
iTaskExecutionEventListener.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);
} }
@Override @Override

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save