Browse Source

[DSIP-73] Add dolphinscheduler-task-executor module to unify the task execution logic (#16790)

dev
Wenjun Ruan 5 days ago committed by GitHub
parent
commit
673687d540
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 15
      docs/docs/en/architecture/configuration.md
  2. 1
      docs/docs/en/guide/upgrade/incompatible.md
  3. 6
      docs/docs/zh/architecture/configuration.md
  4. 1
      docs/docs/zh/guide/upgrade/incompatible.md
  5. 14
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
  6. 7
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
  7. 9
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
  8. 30
      dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEvent.java
  9. 10
      dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java
  10. 10
      dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java
  11. 5
      dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml
  12. 44
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/serialize/JsonSerializer.java
  13. 6
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/NettyRemotingServer.java
  14. 2
      dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml
  15. 5
      dolphinscheduler-extract/dolphinscheduler-extract-common/src/main/java/org/apache/dolphinscheduler/extract/common/ILogService.java
  16. 12
      dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
  17. 45
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskExecutorOperator.java
  18. 41
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskInstanceOperator.java
  19. 50
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutionEventListener.java
  20. 54
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutorEventListener.java
  21. 120
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/TaskExecutorEventRemoteReporterClient.java
  22. 7
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/ITaskExecutionEvent.java
  23. 51
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerRequest.java
  24. 55
      dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionSuccessEvent.java
  25. 8
      dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
  26. 50
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/IPhysicalTaskExecutorOperator.java
  27. 51
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceExecutionEventAckListener.java
  28. 46
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceOperator.java
  29. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionFailedEventAck.java
  30. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionKilledEventAck.java
  31. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionPausedEventAck.java
  32. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionSuccessEventAck.java
  33. 42
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchResponse.java
  34. 39
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionDispatchedEventAck.java
  35. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionFinishEventAck.java
  36. 40
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionRunningEventAck.java
  37. 74
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillResponse.java
  38. 1
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointRequest.java
  39. 1
      dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointResponse.java
  40. 6
      dolphinscheduler-master/pom.xml
  41. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  42. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java
  43. 5
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
  44. 1
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/IWorkflowRepository.java
  45. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java
  46. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowCacheRepository.java
  47. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java
  48. 6
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorker.java
  49. 9
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorkers.java
  50. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/IdSlotBasedCommandFetcher.java
  51. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskReassignMasterHostException.java
  52. 78
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineDelegator.java
  53. 52
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java
  54. 88
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutor.java
  55. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
  56. 45
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerProvider.java
  57. 28
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
  58. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventRemoteReporterClient.java
  59. 55
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorFactory.java
  60. 36
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java
  61. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventReporter.java
  62. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorRepository.java
  63. 74
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/AbstractLogicTask.java
  64. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTask.java
  65. 9
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTaskPluginFactory.java
  66. 6
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java
  67. 14
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/LogicTaskPluginFactoryBuilder.java
  68. 47
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTask.java
  69. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTaskPluginFactory.java
  70. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTask.java
  71. 11
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTaskPluginFactory.java
  72. 25
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentTaskTracker.java
  73. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicAsyncTaskExecuteFunction.java
  74. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicCommandUtils.java
  75. 75
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTask.java
  76. 16
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTaskPluginFactory.java
  77. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicOutput.java
  78. 55
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTask.java
  79. 25
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTaskPluginFactory.java
  80. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowControlClient.java
  81. 49
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTask.java
  82. 34
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskPluginFactory.java
  83. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskRuntimeContext.java
  84. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowTracker.java
  85. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/trigger/SubWorkflowManualTrigger.java
  86. 40
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTask.java
  87. 11
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTaskPluginFactory.java
  88. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/graph/SuccessorFlowAdjuster.java
  89. 18
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/SystemEventBus.java
  90. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/AbstractSystemEvent.java
  91. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java
  92. 33
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java
  93. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java
  94. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskOperator.java
  95. 69
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java
  96. 120
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java
  97. 53
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java
  98. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java
  99. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java
  100. 50
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRuntimeContextChangedEvent.java
  101. Some files were not shown because too many files have changed in this diff Show More

15
docs/docs/en/architecture/configuration.md

@ -277,24 +277,16 @@ Location: `api-server/conf/application.yaml`
Location: `master-server/conf/application.yaml`
| Parameters | Default value | Description |
|-----------------------------------------------------------------------------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
|-----------------------------------------------------------------------------|------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------|
| master.listen-port | 5678 | master listen port |
| master.pre-exec-threads | 10 | master prepare execute thread number to limit handle commands in parallel |
| master.exec-threads | 100 | master execute thread number to limit process instances in parallel |
| master.dispatch-task-number | 3 | master dispatch task number per batch |
| master.logic-task-config.task-executor-thread-count | 2 * CPU +1 | The thread size used to execute logic task |
| master.worker-load-balancer-configuration-properties.type | DYNAMIC_WEIGHTED_ROUND_ROBIN | Master will use the worker's cpu/memory/threadPool usage to calculate the worker load, the lower load will have more change to be dispatched task |
| master.max-heartbeat-interval | 10s | master max heartbeat interval |
| master.task-commit-retry-times | 5 | master commit task retry times |
| master.task-commit-interval | 1000 | master commit task interval, the unit is millisecond |
| master.state-wheel-interval | 5 | time to check status |
| master.server-load-protection.enabled | true | If set true, will open master overload protection |
| master.server-load-protection.max-system-cpu-usage-percentage-thresholds | 0.7 | Master max system cpu usage, when the master's system cpu usage is smaller then this value, master server can execute workflow. |
| master.server-load-protection.max-jvm-cpu-usage-percentage-thresholds | 0.7 | Master max JVM cpu usage, when the master's jvm cpu usage is smaller then this value, master server can execute workflow. |
| master.server-load-protection.max-system-memory-usage-percentage-thresholds | 0.7 | Master max system memory usage , when the master's system memory usage is smaller then this value, master server can execute workflow. |
| master.server-load-protection.max-disk-usage-percentage-thresholds | 0.7 | Master max disk usage , when the master's disk usage is smaller then this value, master server can execute workflow. |
| master.failover-interval | 10 | failover interval, the unit is minute |
| master.kill-application-when-task-failover | true | whether to kill yarn/k8s application when failover taskInstance |
| master.registry-disconnect-strategy.max-waiting-time | 100s | Used when the master disconnect from registry, and the disconnect strategy is waiting, this config means the master will waiting to reconnect to registry in given times, and after the waiting times, if the master still cannot connect to registry, will stop itself, if the value is 0s, the Master will wait infinitely |
| master.worker-group-refresh-interval | 10s | The interval to refresh worker group from db to memory |
| master.command-fetch-strategy.type | ID_SLOT_BASED | The command fetch strategy, only support `ID_SLOT_BASED` |
| master.command-fetch-strategy.config.id-step | 1 | The id auto incremental step of t_ds_command in db |
@ -307,7 +299,6 @@ Location: `worker-server/conf/application.yaml`
| Parameters | Default value | Description |
|-----------------------------------------------------------------------------|---------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| worker.listen-port | 1234 | worker-service listen port |
| worker.exec-threads | 100 | worker-service execute thread number, used to limit the number of task instances in parallel |
| worker.max-heartbeat-interval | 10s | worker-service max heartbeat interval |
| worker.host-weight | 100 | worker host weight to dispatch tasks |
| worker.server-load-protection.enabled | true | If set true will open worker overload protection |
@ -317,7 +308,7 @@ Location: `worker-server/conf/application.yaml`
| worker.server-load-protection.max-disk-usage-percentage-thresholds | 0.7 | Worker max disk usage , when the worker's disk usage is smaller then this value, master server can execute workflow. |
| worker.registry-disconnect-strategy.strategy | stop | Used when the worker disconnect from registry, default value: stop. Optional values include stop, waiting |
| worker.registry-disconnect-strategy.max-waiting-time | 100s | Used when the worker disconnect from registry, and the disconnect strategy is waiting, this config means the worker will waiting to reconnect to registry in given times, and after the waiting times, if the worker still cannot connect to registry, will stop itself, if the value is 0s, will wait infinitely |
| worker.task-execute-threads-full-policy | REJECT | If REJECT, when the task waiting in the worker reaches exec-threads, it will reject the received task and the Master will redispatch it; If CONTINUE, it will put the task into the worker's execution queue and wait for a free thread to start execution |
| worker.physical-task-config.task-executor-thread-size | 100 | The thread size used to execute physical task |
| worker.tenant-config.auto-create-tenant-enabled | true | tenant corresponds to the user of the system, which is used by the worker to submit the job. If system does not have this user, it will be automatically created after the parameter worker.tenant.auto.create is true. |
| worker.tenant-config.default-tenant-enabled | false | If set true, will use worker bootstrap user as the tenant to execute task when the tenant is `default`. |

1
docs/docs/en/guide/upgrade/incompatible.md

@ -34,4 +34,5 @@ This document records the incompatible updates between each version. You need to
* Deprecated upgrade code of 1.x and 2.x ([#16543])(https://github.com/apache/dolphinscheduler/pull/16543)
* Remove the `Data Quality` module ([#16794])(https://github.com/apache/dolphinscheduler/pull/16794)
* Remove the `registry-disconnect-strategy` in `application.yaml` ([#16821])(https://github.com/apache/dolphinscheduler/pull/16821)
* Remove `exec-threads` in worker's `application.yaml`, please use `physical-task-config`;Remove `master-async-task-executor-thread-pool-size` in master's `application.yaml`, please use `logic-task-config` ([#16790])(https://github.com/apache/dolphinscheduler/pull/16790)

6
docs/docs/zh/architecture/configuration.md

@ -304,9 +304,8 @@ common.properties配置文件目前主要是配置hadoop/s3/yarn/applicationId
位置:`worker-server/conf/application.yaml`
| 参数 | 默认值 | 描述 |
|-----------------------------------------------------------------------------|-----------|-------------------------------------------------------------------------------------------------------------------------------------------|
|-----------------------------------------------------------------------------|-----------|-----------------------------------------------------------------------------------------|
| worker.listen-port | 1234 | worker监听端口 |
| worker.exec-threads | 100 | worker工作线程数量,用于限制并行的任务实例数量 |
| worker.max-heartbeat-interval | 10s | worker最大心跳间隔 |
| worker.host-weight | 100 | 派发任务时,worker主机的权重 |
| worker.tenant-auto-create | true | 租户对应于系统的用户,由worker提交作业.如果系统没有该用户,则在参数worker.tenant.auto.create为true后自动创建。 |
@ -317,8 +316,7 @@ common.properties配置文件目前主要是配置hadoop/s3/yarn/applicationId
| worker.server-load-protection.max-disk-usage-percentage-thresholds | 0.7 | worker最大系统磁盘使用值,只有当前系统磁盘使用值低于最大系统磁盘使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的操作系统磁盘空间 |
| worker.alert-listen-host | localhost | alert监听host |
| worker.alert-listen-port | 50052 | alert监听端口 |
| worker.registry-disconnect-strategy.max-waiting-time | 100s | 当Worker与注册中心失联之后重连时间, 之后当strategy为waiting时,该值生效。 该值表示当Worker与注册中心失联时会在给定时间之内进行重连, 在给定时间之内重连失败将会停止自己,在重连时,Worker会丢弃kill正在执行的任务。值为0表示会无限期等待 |
| worker.task-execute-threads-full-policy | REJECT | 如果是 REJECT, 当Worker中等待队列中的任务数达到exec-threads时, Worker将会拒绝接下来新接收的任务,Master将会重新分发该任务; 如果是 CONTINUE, Worker将会接收任务,放入等待队列中等待空闲线程去执行该任务 |
| worker.physical-task-config.task-executor-thread-size | 100 | Worker中任务最大并发度 |
| worker.tenant-config.auto-create-tenant-enabled | true | 租户对应于系统的用户,由worker提交作业.如果系统没有该用户,则在参数worker.tenant.auto.create为true后自动创建。 |
| worker.tenant-config.default-tenant-enabled | false | 如果设置为true, 将会使用worker服务启动用户作为 `default` 租户。 |

1
docs/docs/zh/guide/upgrade/incompatible.md

@ -32,4 +32,5 @@
* 废弃从 1.x 至 2.x 的升级代码 ([#16543])(https://github.com/apache/dolphinscheduler/pull/16543)
* 移除 `数据质量` 模块 ([#16794])(https://github.com/apache/dolphinscheduler/pull/16794)
* 在`application.yaml`中移除`registry-disconnect-strategy`配置 ([#16821])(https://github.com/apache/dolphinscheduler/pull/16821)
* 在worker的`application.yaml`中移除`exec-threads`,使用`physical-task-config`替代;在master的`application.yaml`中移除`master-async-task-executor-thread-pool-size`使用`logic-task-config`替代 ([#16790])(https://github.com/apache/dolphinscheduler/pull/16790)

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

@ -46,14 +46,14 @@ import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils;
import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.common.ILogService;
import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator;
import org.apache.dolphinscheduler.extract.worker.IStreamingTaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceTriggerSavepointRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceTriggerSavepointResponse;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
@ -302,11 +302,11 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
}
// todo: we only support streaming task for now
final TaskInstanceKillResponse taskInstanceKillResponse = Clients
.withService(ITaskInstanceOperator.class)
final TaskExecutorKillResponse taskExecutorKillResponse = Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.killTask(new TaskInstanceKillRequest(taskInstanceId));
log.info("TaskInstance kill response: {}", taskInstanceKillResponse);
.killTask(TaskExecutorKillRequest.of(taskInstanceId));
log.info("TaskInstance kill response: {}", taskExecutorKillResponse);
putMsg(result, Status.SUCCESS);
return result;

7
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java

@ -43,8 +43,6 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.extract.base.config.NettyServerConfig;
import org.apache.dolphinscheduler.extract.base.server.SpringServerMethodInvokerDiscovery;
import org.apache.dolphinscheduler.extract.common.ILogService;
import org.apache.dolphinscheduler.extract.common.transportor.GetAppIdRequest;
import org.apache.dolphinscheduler.extract.common.transportor.GetAppIdResponse;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadRequest;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadResponse;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogPageQueryRequest;
@ -131,11 +129,6 @@ public class LoggerServiceTest {
return new TaskInstanceLogPageQueryResponse();
}
@Override
public GetAppIdResponse getAppId(GetAppIdRequest getAppIdRequest) {
return new GetAppIdResponse();
}
@Override
public void removeTaskInstanceLog(String taskInstanceLogAbsolutePath) {

9
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java

@ -31,6 +31,12 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
@Slf4j
public class ThreadUtils {
/**
* Create a daemon fixed thread pool, the thread name will be formatted with the given name.
*
* @param threadNameFormat the thread name format, e.g. "DemonThread-%d"
* @param threadsNum the number of threads in the pool
*/
public static ThreadPoolExecutor newDaemonFixedThreadExecutor(String threadNameFormat, int threadsNum) {
return (ThreadPoolExecutor) Executors.newFixedThreadPool(threadsNum, newDaemonThreadFactory(threadNameFormat));
}
@ -45,7 +51,8 @@ public class ThreadUtils {
* @param threadNameFormat the thread name format, e.g. "DemonThread-%d"
* @param threadsNum the number of threads in the pool
*/
public static ScheduledExecutorService newDaemonScheduledExecutorService(String threadNameFormat, int threadsNum) {
public static ScheduledExecutorService newDaemonScheduledExecutorService(final String threadNameFormat,
final int threadsNum) {
return Executors.newScheduledThreadPool(threadsNum, newDaemonThreadFactory(threadNameFormat));
}

30
dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEvent.java

@ -20,37 +20,47 @@ package org.apache.dolphinscheduler.eventbus;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
import lombok.Builder;
import lombok.ToString;
import lombok.experimental.SuperBuilder;
/**
* The abstract class of delay event, the event will be triggered after the delay time.
* <p> You can extend this class to implement your own delay event.
*/
@ToString
@SuperBuilder
public abstract class AbstractDelayEvent implements IEvent, Delayed {
private static final long DEFAULT_DELAY_TIME = 0;
protected long delayTime;
protected long triggerTimeInMillis;
@Builder.Default
protected long createTimeInNano = System.nanoTime();
public AbstractDelayEvent() {
this(0);
this(DEFAULT_DELAY_TIME);
}
public AbstractDelayEvent(long delayTime) {
if (delayTime == 0) {
this.triggerTimeInMillis = System.currentTimeMillis();
} else {
this.triggerTimeInMillis = System.currentTimeMillis() + delayTime;
public AbstractDelayEvent(final long delayTime) {
this(delayTime, System.nanoTime());
}
public AbstractDelayEvent(final long delayTime, final long createTimeInNano) {
this.delayTime = delayTime;
this.createTimeInNano = createTimeInNano;
}
@Override
public long getDelay(TimeUnit unit) {
long delay = triggerTimeInMillis - System.currentTimeMillis();
return unit.convert(delay, TimeUnit.MILLISECONDS);
long delay = createTimeInNano + delayTime * 1_000_000 - System.nanoTime();
return unit.convert(delay, TimeUnit.NANOSECONDS);
}
@Override
public int compareTo(Delayed other) {
return Long.compare(this.triggerTimeInMillis, ((AbstractDelayEvent) other).triggerTimeInMillis);
return Long.compare(this.createTimeInNano, ((AbstractDelayEvent) other).createTimeInNano);
}
}

10
dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java

@ -37,6 +37,16 @@ public abstract class AbstractDelayEventBus<T extends AbstractDelayEvent> implem
return Optional.ofNullable(delayEventQueue.poll());
}
@Override
public Optional<T> peek() {
return Optional.ofNullable(delayEventQueue.peek());
}
@Override
public Optional<T> remove() {
return Optional.ofNullable(delayEventQueue.remove());
}
@Override
public boolean isEmpty() {
return delayEventQueue.isEmpty();

10
dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java

@ -54,6 +54,16 @@ public interface IEventBus<T extends IEvent> {
*/
Optional<T> poll() throws InterruptedException;
/**
* peek the head event from the bus. This method will not block if the event bus is empty will return empty optional.
*/
Optional<T> peek();
/**
* Remove the head event from the bus. This method will not block if the event bus is empty will return empty optional.
*/
Optional<T> remove();
/**
* Whether the bus is empty.
*/

5
dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml

@ -45,27 +45,28 @@
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-meter</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>org.springframework</groupId>
<artifactId>spring-context</artifactId>

44
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/serialize/JsonSerializer.java

@ -25,12 +25,12 @@ import static org.apache.dolphinscheduler.common.constants.DateConstants.YYYY_MM
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.text.SimpleDateFormat;
import java.time.LocalDateTime;
import java.util.TimeZone;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.JsonProcessingException;
@ -38,9 +38,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
/**
* json serialize or deserialize
*/
@Slf4j
public class JsonSerializer {
@ -60,13 +57,6 @@ public class JsonSerializer {
}
/**
* serialize to byte
*
* @param obj object
* @param <T> object type
* @return byte array
*/
public static <T> byte[] serialize(T obj) {
if (obj == null) {
return null;
@ -79,44 +69,14 @@ public class JsonSerializer {
}
}
/**
* serialize to string
*
* @param obj object
* @param <T> object type
* @return string
*/
public static <T> String serializeToString(T obj) {
String json = "";
try {
json = objectMapper.writeValueAsString(obj);
} catch (JsonProcessingException e) {
log.error("serializeToString exception!", e);
}
return json;
}
/**
* deserialize
*
* @param src byte array
* @param clazz class
* @param <T> deserialize type
* @return deserialize type
*/
@SneakyThrows
public static <T> T deserialize(byte[] src, Class<T> clazz) {
if (src == null) {
return null;
}
String json = new String(src, StandardCharsets.UTF_8);
try {
return objectMapper.readValue(json, clazz);
} catch (IOException e) {
log.error("deserialize exception!", e);
return null;
}
}
}

6
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/NettyRemotingServer.java

@ -70,12 +70,12 @@ class NettyRemotingServer {
this.serverConfig = serverConfig;
this.serverName = serverConfig.getServerName();
this.methodInvokerExecutor = ThreadUtils.newDaemonFixedThreadExecutor(
serverName + "MethodInvoker-%d", Runtime.getRuntime().availableProcessors() * 2 + 1);
serverName + "-methodInvoker-%d", Runtime.getRuntime().availableProcessors() * 2 + 1);
this.channelHandler = new JdkDynamicServerHandler(methodInvokerExecutor);
ThreadFactory bossThreadFactory =
ThreadUtils.newDaemonThreadFactory(serverName + "BossThread-%d");
ThreadUtils.newDaemonThreadFactory(serverName + "-boss-%d");
ThreadFactory workerThreadFactory =
ThreadUtils.newDaemonThreadFactory(serverName + "WorkerThread-%d");
ThreadUtils.newDaemonThreadFactory(serverName + "-worker-%d");
if (Epoll.isAvailable()) {
this.bossGroup = new EpollEventLoopGroup(1, bossThreadFactory);
this.workGroup = new EpollEventLoopGroup(serverConfig.getWorkerThread(), workerThreadFactory);

2
dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml

@ -27,14 +27,12 @@
<version>dev-SNAPSHOT</version>
</parent>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-common</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-base</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>

5
dolphinscheduler-extract/dolphinscheduler-extract-common/src/main/java/org/apache/dolphinscheduler/extract/common/ILogService.java

@ -19,8 +19,6 @@ package org.apache.dolphinscheduler.extract.common;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.common.transportor.GetAppIdRequest;
import org.apache.dolphinscheduler.extract.common.transportor.GetAppIdResponse;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadRequest;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogFileDownloadResponse;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogPageQueryRequest;
@ -35,9 +33,6 @@ public interface ILogService {
@RpcMethod
TaskInstanceLogPageQueryResponse pageQueryTaskInstanceLog(TaskInstanceLogPageQueryRequest taskInstanceLogPageQueryRequest);
@RpcMethod
GetAppIdResponse getAppId(GetAppIdRequest getAppIdRequest);
@RpcMethod
void removeTaskInstanceLog(String taskInstanceLogAbsolutePath);

12
dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml

@ -30,21 +30,27 @@
<artifactId>dolphinscheduler-extract-master</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-base</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-task-api</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-task-executor</artifactId>
</dependency>
</dependencies>
</project>

45
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskExecutorOperator.java

@ -0,0 +1,45 @@
/*
* 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.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse;
@RpcService
public interface ILogicTaskExecutorOperator {
@RpcMethod
TaskExecutorDispatchResponse dispatchTask(final TaskExecutorDispatchRequest taskExecutorDispatchRequest);
@RpcMethod
TaskExecutorKillResponse killTask(final TaskExecutorKillRequest taskExecutorKillRequest);
@RpcMethod
TaskExecutorPauseResponse pauseTask(final TaskExecutorPauseRequest taskExecutorPauseRequest);
@RpcMethod
void ackTaskExecutorLifecycleEvent(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck);
}

41
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskInstanceOperator.java

@ -1,41 +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.LogicTaskDispatchRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchResponse;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseResponse;
@RpcService
public interface ILogicTaskInstanceOperator {
@RpcMethod
LogicTaskDispatchResponse dispatchLogicTask(LogicTaskDispatchRequest taskDispatchRequest);
@RpcMethod
LogicTaskKillResponse killLogicTask(LogicTaskKillRequest taskKillRequest);
@RpcMethod
LogicTaskPauseResponse pauseLogicTask(LogicTaskPauseRequest taskPauseRequest);
}

50
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutionEventListener.java

@ -1,50 +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.TaskExecutionDispatchEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent;
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent;
@RpcService
public interface ITaskExecutionEventListener {
@RpcMethod
void onTaskInstanceDispatched(final TaskExecutionDispatchEvent taskExecutionDispatchEvent);
@RpcMethod
void onTaskInstanceExecutionRunning(final TaskExecutionRunningEvent taskInstanceExecutionRunningEvent);
@RpcMethod
void onTaskInstanceExecutionSuccess(final TaskExecutionSuccessEvent taskInstanceExecutionSuccessEvent);
@RpcMethod
void onTaskInstanceExecutionFailed(final TaskExecutionFailedEvent taskInstanceExecutionFailedEvent);
@RpcMethod
void onTaskInstanceExecutionKilled(final TaskExecutionKilledEvent taskInstanceExecutionKilledEvent);
@RpcMethod
void onTaskInstanceExecutionPaused(final TaskExecutionPausedEvent taskInstanceExecutionPausedEvent);
}

54
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutorEventListener.java

@ -0,0 +1,54 @@
/*
* 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.task.executor.events.TaskExecutorDispatchedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorFailedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorKilledLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorPausedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorRuntimeContextChangedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorStartedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorSuccessLifecycleEvent;
@RpcService
public interface ITaskExecutorEventListener {
@RpcMethod
void onTaskExecutorDispatched(final TaskExecutorDispatchedLifecycleEvent taskExecutorDispatchedLifecycleEvent);
@RpcMethod
void onTaskExecutorRunning(final TaskExecutorStartedLifecycleEvent taskExecutorStartedLifecycleEvent);
@RpcMethod
void onTaskExecutorRuntimeContextChanged(final TaskExecutorRuntimeContextChangedLifecycleEvent taskExecutorRuntimeContextChangedLifecycleEventr);
@RpcMethod
void onTaskExecutorSuccess(final TaskExecutorSuccessLifecycleEvent taskExecutorSuccessLifecycleEvent);
@RpcMethod
void onTaskExecutorFailed(final TaskExecutorFailedLifecycleEvent taskExecutorFailedLifecycleEvent);
@RpcMethod
void onTaskExecutorKilled(final TaskExecutorKilledLifecycleEvent taskExecutorKilledLifecycleEvent);
@RpcMethod
void onTaskExecutorPaused(final TaskExecutorPausedLifecycleEvent taskExecutorPausedLifecycleEvent);
}

120
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/TaskExecutorEventRemoteReporterClient.java

@ -0,0 +1,120 @@
/*
* 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.client.Clients;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorEventRemoteReporterClient;
import org.apache.dolphinscheduler.task.executor.events.IReportableTaskExecutorLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorDispatchedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorFailedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorKilledLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorPausedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorRuntimeContextChangedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorStartedLifecycleEvent;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorSuccessLifecycleEvent;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class TaskExecutorEventRemoteReporterClient implements ITaskExecutorEventRemoteReporterClient {
public void reportTaskExecutionEventToMaster(final IReportableTaskExecutorLifecycleEvent taskExecutorLifecycleEvent) {
try {
taskExecutorLifecycleEvent.setLatestReportTime(System.currentTimeMillis());
switch (taskExecutorLifecycleEvent.getType()) {
case DISPATCHED:
reportTaskDispatchedEventToMaster(
(TaskExecutorDispatchedLifecycleEvent) taskExecutorLifecycleEvent);
break;
case RUNNING:
reportTaskRunningEventToMaster((TaskExecutorStartedLifecycleEvent) taskExecutorLifecycleEvent);
break;
case RUNTIME_CONTEXT_CHANGE:
reportTaskRuntimeContextChangeEventToMaster(
(TaskExecutorRuntimeContextChangedLifecycleEvent) taskExecutorLifecycleEvent);
break;
case PAUSED:
reportTaskPausedEventToMaster((TaskExecutorPausedLifecycleEvent) taskExecutorLifecycleEvent);
break;
case KILLED:
reportTaskKilledEventToMaster((TaskExecutorKilledLifecycleEvent) taskExecutorLifecycleEvent);
break;
case FAILED:
reportTaskFailedEventToMaster((TaskExecutorFailedLifecycleEvent) taskExecutorLifecycleEvent);
break;
case SUCCESS:
reportTaskSuccessEventToMaster((TaskExecutorSuccessLifecycleEvent) taskExecutorLifecycleEvent);
break;
default:
log.warn("Unsupported TaskExecutionEvent: {}", taskExecutorLifecycleEvent);
}
log.info("Report: {} to master success", taskExecutorLifecycleEvent);
} catch (Throwable throwable) {
log.error("Report ITaskExecutorLifecycleEvent: {} to master failed", taskExecutorLifecycleEvent, throwable);
}
}
private static void reportTaskDispatchedEventToMaster(final TaskExecutorDispatchedLifecycleEvent taskExecutionDispatchedEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionDispatchedEvent.getWorkflowInstanceHost())
.onTaskExecutorDispatched(taskExecutionDispatchedEvent);
}
private static void reportTaskRunningEventToMaster(final TaskExecutorStartedLifecycleEvent taskExecutionRunningEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionRunningEvent.getWorkflowInstanceHost())
.onTaskExecutorRunning(taskExecutionRunningEvent);
}
private static void reportTaskRuntimeContextChangeEventToMaster(final TaskExecutorRuntimeContextChangedLifecycleEvent taskExecutorLifecycleEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutorLifecycleEvent.getWorkflowInstanceHost())
.onTaskExecutorRuntimeContextChanged(taskExecutorLifecycleEvent);
}
private static void reportTaskPausedEventToMaster(final TaskExecutorPausedLifecycleEvent taskExecutionPausedEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionPausedEvent.getWorkflowInstanceHost())
.onTaskExecutorPaused(taskExecutionPausedEvent);
}
private static void reportTaskKilledEventToMaster(final TaskExecutorKilledLifecycleEvent taskExecutionKilledEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionKilledEvent.getWorkflowInstanceHost())
.onTaskExecutorKilled(taskExecutionKilledEvent);
}
private static void reportTaskFailedEventToMaster(final TaskExecutorFailedLifecycleEvent taskExecutionFailedEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionFailedEvent.getWorkflowInstanceHost())
.onTaskExecutorFailed(taskExecutionFailedEvent);
}
private static void reportTaskSuccessEventToMaster(final TaskExecutorSuccessLifecycleEvent taskExecutionSuccessEvent) {
Clients
.withService(ITaskExecutorEventListener.class)
.withHost(taskExecutionSuccessEvent.getWorkflowInstanceHost())
.onTaskExecutorSuccess(taskExecutionSuccessEvent);
}
}

7
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/ITaskExecutionEvent.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.extract.master.transportor;
// todo: add event source to distinguish the event is from executor or user operation
// 将这个类移到common中?
public interface ITaskExecutionEvent {
int getWorkflowInstanceId();
@ -28,9 +29,9 @@ public interface ITaskExecutionEvent {
void setEventCreateTime(long eventCreateTime);
long getEventSendTime();
Long getEventSendTime();
void setEventSendTime(long eventSendTime);
void setEventSendTime(Long eventSendTime);
void setWorkflowInstanceHost(String host);
@ -43,7 +44,7 @@ public interface ITaskExecutionEvent {
TaskInstanceExecutionEventType getEventType();
enum TaskInstanceExecutionEventType {
DISPATCH,
DISPATCHED,
RUNNING,
PAUSED,
KILLED,

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

@ -1,51 +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.transportor;
import java.util.Map;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
public class StreamingTaskTriggerRequest {
private int executorId;
private String executorName;
private long projectCode;
private long taskDefinitionCode;
private int taskDefinitionVersion;
private int warningGroupId;
private String workerGroup;
private Long environmentCode;
private Map<String, String> startParams;
private String tenantCode;
private int dryRun;
}

55
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionSuccessEvent.java

@ -1,55 +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.transportor;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionSuccessEvent implements ITaskExecutionEvent {
private int taskInstanceId;
private int workflowInstanceId;
private String taskInstanceHost;
private String workflowInstanceHost;
private long endTime;
private int processId;
private String appIds;
private String varPool;
private long eventCreateTime;
private long eventSendTime;
@Override
public TaskInstanceExecutionEventType getEventType() {
return TaskInstanceExecutionEventType.SUCCESS;
}
}

8
dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml

@ -30,11 +30,12 @@
<artifactId>dolphinscheduler-extract-worker</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-base</artifactId>
@ -45,6 +46,11 @@
<artifactId>dolphinscheduler-task-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-task-executor</artifactId>
</dependency>
</dependencies>
</project>

50
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/IPhysicalTaskExecutorOperator.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.worker;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterResponse;
@RpcService
public interface IPhysicalTaskExecutorOperator {
@RpcMethod
TaskExecutorDispatchResponse dispatchTask(final TaskExecutorDispatchRequest taskExecutorDispatchRequest);
@RpcMethod
TaskExecutorKillResponse killTask(final TaskExecutorKillRequest taskExecutorKillRequest);
@RpcMethod
TaskExecutorPauseResponse pauseTask(final TaskExecutorPauseRequest taskExecutorPauseRequest);
@RpcMethod
TaskExecutorReassignMasterResponse reassignWorkflowInstanceHost(final TaskExecutorReassignMasterRequest taskExecutorReassignMasterRequest);
@RpcMethod
void ackPhysicalTaskExecutorLifecycleEvent(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck);
}

51
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceExecutionEventAckListener.java

@ -1,51 +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.worker;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskExecutionFailedEventAck;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskExecutionKilledEventAck;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskExecutionPausedEventAck;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskExecutionSuccessEventAck;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionDispatchedEventAck;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceExecutionRunningEventAck;
@RpcService
public interface ITaskInstanceExecutionEventAckListener {
@RpcMethod
void handleTaskInstanceDispatchedEventAck(final TaskInstanceExecutionDispatchedEventAck taskInstanceExecutionDispatchedEventAck);
// todo: If we use sync, then we don't need ack here
@RpcMethod
void handleTaskInstanceExecutionRunningEventAck(TaskInstanceExecutionRunningEventAck taskInstanceExecutionRunningEventAck);
@RpcMethod
void handleTaskExecutionSuccessEventAck(TaskExecutionSuccessEventAck taskExecutionSuccessEventAck);
@RpcMethod
void handleTaskExecutionPausedEventAck(TaskExecutionPausedEventAck taskExecutionPausedEventAck);
@RpcMethod
void handleTaskExecutionFailedEventAck(TaskExecutionFailedEventAck taskExecutionFailedEventAck);
@RpcMethod
void handleTaskExecutionKilledEventAck(TaskExecutionKilledEventAck taskExecutionKilledEventAck);
}

46
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceOperator.java

@ -1,46 +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.worker;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskResponse;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchResponse;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseResponse;
@RpcService
public interface ITaskInstanceOperator {
@RpcMethod
TaskInstanceDispatchResponse dispatchTask(final TaskInstanceDispatchRequest taskInstanceDispatchRequest);
@RpcMethod
TaskInstanceKillResponse killTask(final TaskInstanceKillRequest taskInstanceKillRequest);
@RpcMethod
TaskInstancePauseResponse pauseTask(final TaskInstancePauseRequest taskPauseRequest);
@RpcMethod
TakeOverTaskResponse takeOverTask(final TakeOverTaskRequest takeOverTaskRequest);
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionFailedEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionFailedEventAck {
private int taskInstanceId;
private boolean success;
public static TaskExecutionFailedEventAck success(int taskInstanceId) {
return new TaskExecutionFailedEventAck(taskInstanceId, true);
}
public static TaskExecutionFailedEventAck failed(int taskInstanceId) {
return new TaskExecutionFailedEventAck(taskInstanceId, false);
}
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionKilledEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionKilledEventAck {
private int taskInstanceId;
private boolean success;
public static TaskExecutionKilledEventAck success(int taskInstanceId) {
return new TaskExecutionKilledEventAck(taskInstanceId, true);
}
public static TaskExecutionKilledEventAck failed(int taskInstanceId) {
return new TaskExecutionKilledEventAck(taskInstanceId, false);
}
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionPausedEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionPausedEventAck {
private int taskInstanceId;
private boolean success;
public static TaskExecutionPausedEventAck success(int taskInstanceId) {
return new TaskExecutionPausedEventAck(taskInstanceId, true);
}
public static TaskExecutionPausedEventAck failed(int taskInstanceId) {
return new TaskExecutionPausedEventAck(taskInstanceId, false);
}
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionSuccessEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionSuccessEventAck {
private int taskInstanceId;
private boolean success;
public static TaskExecutionSuccessEventAck success(int taskInstanceId) {
return new TaskExecutionSuccessEventAck(taskInstanceId, true);
}
public static TaskExecutionSuccessEventAck failed(int taskInstanceId) {
return new TaskExecutionSuccessEventAck(taskInstanceId, false);
}
}

42
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchResponse.java

@ -1,42 +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.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskInstanceDispatchResponse {
private Integer taskInstanceId;
private boolean dispatchSuccess;
private String message;
public static TaskInstanceDispatchResponse success(Integer taskInstanceId) {
return new TaskInstanceDispatchResponse(taskInstanceId, true, "dispatch success");
}
public static TaskInstanceDispatchResponse failed(Integer taskInstanceId, String message) {
return new TaskInstanceDispatchResponse(taskInstanceId, false, message);
}
}

39
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionDispatchedEventAck.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.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskInstanceExecutionDispatchedEventAck {
private int taskInstanceId;
private boolean success;
public static TaskInstanceExecutionDispatchedEventAck success(int taskInstanceId) {
return new TaskInstanceExecutionDispatchedEventAck(taskInstanceId, true);
}
public static TaskInstanceExecutionDispatchedEventAck failed(int taskInstanceId) {
return new TaskInstanceExecutionDispatchedEventAck(taskInstanceId, false);
}
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionFinishEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskInstanceExecutionFinishEventAck {
private int taskInstanceId;
private boolean success;
public static TaskInstanceExecutionFinishEventAck success(int taskInstanceId) {
return new TaskInstanceExecutionFinishEventAck(taskInstanceId, true);
}
public static TaskInstanceExecutionFinishEventAck failed(int taskInstanceId) {
return new TaskInstanceExecutionFinishEventAck(taskInstanceId, false);
}
}

40
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionRunningEventAck.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.extract.worker.transportor;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskInstanceExecutionRunningEventAck {
private int taskInstanceId;
private boolean success;
public static TaskInstanceExecutionRunningEventAck success(int taskInstanceId) {
return new TaskInstanceExecutionRunningEventAck(taskInstanceId, true);
}
public static TaskInstanceExecutionRunningEventAck failed(int taskInstanceId) {
return new TaskInstanceExecutionRunningEventAck(taskInstanceId, false);
}
}

74
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillResponse.java

@ -1,74 +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.worker.transportor;
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import java.util.Arrays;
import java.util.List;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@AllArgsConstructor
public class TaskInstanceKillResponse {
private int taskInstanceId;
private String host;
private TaskExecutionStatus status;
private int processId;
/**
* other resource manager appId , for example : YARN etc
*/
private List<String> appIds;
private boolean success;
private String message;
public static TaskInstanceKillResponse success(TaskExecutionContext taskExecutionContext) {
TaskInstanceKillResponse taskInstanceKillResponse = new TaskInstanceKillResponse();
taskInstanceKillResponse.setSuccess(true);
taskInstanceKillResponse.setStatus(taskExecutionContext.getCurrentExecutionStatus());
if (taskExecutionContext.getAppIds() != null) {
taskInstanceKillResponse
.setAppIds(Arrays.asList(taskExecutionContext.getAppIds().split(TaskConstants.COMMA)));
}
taskInstanceKillResponse.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
taskInstanceKillResponse.setHost(taskExecutionContext.getHost());
taskInstanceKillResponse.setProcessId(taskExecutionContext.getProcessId());
return taskInstanceKillResponse;
}
public static TaskInstanceKillResponse fail(String message) {
TaskInstanceKillResponse taskInstanceKillResponse = new TaskInstanceKillResponse();
taskInstanceKillResponse.setSuccess(false);
taskInstanceKillResponse.setMessage(message);
return taskInstanceKillResponse;
}
}

1
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointRequest.java

@ -21,6 +21,7 @@ import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Deprecated
@Data
@NoArgsConstructor
@AllArgsConstructor

1
dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointResponse.java

@ -21,6 +21,7 @@ import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
@Deprecated
@Data
@NoArgsConstructor
@AllArgsConstructor

6
dolphinscheduler-master/pom.xml

@ -107,7 +107,11 @@
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-eventbus</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-task-executor</artifactId>
</dependency>
<dependency>

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

@ -40,6 +40,7 @@ import org.apache.dolphinscheduler.server.master.engine.system.event.GlobalMaste
import org.apache.dolphinscheduler.server.master.metrics.MasterServerMetrics;
import org.apache.dolphinscheduler.server.master.registry.MasterRegistryClient;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcServer;
import org.apache.dolphinscheduler.server.master.utils.MasterThreadFactory;
import org.apache.dolphinscheduler.service.ServiceConfiguration;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
@ -166,6 +167,7 @@ public class MasterServer implements IStoppable {
}
// thread sleep 3 seconds for thread quietly stop
ThreadUtils.sleep(Constants.SERVER_CLOSE_WAIT_TIME.toMillis());
MasterThreadFactory.getDefaultSchedulerThreadExecutor().shutdownNow();
try (
SystemEventBusFireWorker systemEventBusFireWorker1 = systemEventBusFireWorker;
WorkflowEngine workflowEngine1 = workflowEngine;
@ -176,7 +178,7 @@ public class MasterServer implements IStoppable {
// like ServerNodeManager,HostManager,TaskResponseService,CuratorZookeeperClient,etc
SpringApplicationContext closedSpringContext = springApplicationContext) {
log.info("Master server is stopping, current cause : {}", cause);
log.info("MasterServer is stopping, current cause : {}", cause);
} catch (Exception e) {
log.error("MasterServer stop failed, current cause: {}", cause, e);
return;

12
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java

@ -15,17 +15,19 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor;
package org.apache.dolphinscheduler.server.master.config;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@NoArgsConstructor
@Builder
@AllArgsConstructor
public class LogicTaskKillRequest {
private int taskInstanceId;
@NoArgsConstructor
public class LogicTaskConfig {
@Builder.Default
private int taskExecutorThreadCount = Runtime.getRuntime().availableProcessors() * 2 + 1;
}

5
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java

@ -48,10 +48,8 @@ public class MasterConfig implements Validator {
private int workflowEventBusFireThreadCount = Runtime.getRuntime().availableProcessors() * 2 + 1;
// todo: change to sync thread pool/ async thread pool ?
private int masterSyncTaskExecutorThreadPoolSize = Runtime.getRuntime().availableProcessors();
private LogicTaskConfig logicTaskConfig = new LogicTaskConfig();
private int masterAsyncTaskExecutorThreadPoolSize = Runtime.getRuntime().availableProcessors();
/**
* Master heart beat task execute interval.
*/
@ -115,6 +113,7 @@ public class MasterConfig implements Validator {
"\n****************************Master Configuration**************************************" +
"\n listen-port -> " + listenPort +
"\n workflow-event-bus-fire-thread-count -> " + workflowEventBusFireThreadCount +
"\n logic-task-config -> " + logicTaskConfig +
"\n max-heartbeat-interval -> " + maxHeartbeatInterval +
"\n server-load-protection -> " + serverLoadProtection +
"\n master-address -> " + masterAddress +

1
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/IWorkflowRepository.java

@ -35,5 +35,4 @@ public interface IWorkflowRepository {
void remove(int workflowInstanceId);
void clear();
}

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

@ -98,7 +98,7 @@ public class TaskGroupCoordinator extends BaseDaemonThread implements AutoClosea
private boolean flag = true;
private static int DEFAULT_LIMIT = 1000;
private static final int DEFAULT_LIMIT = 1000;
public TaskGroupCoordinator() {
super("TaskGroupCoordinator");
@ -126,7 +126,7 @@ public class TaskGroupCoordinator extends BaseDaemonThread implements AutoClosea
dealWithWaitingTaskGroupQueue();
taskGroupCoordinatorRoundCost.stop();
log.info("TaskGroupCoordinator round cost: {}/ms", taskGroupCoordinatorRoundCost.getTime());
log.debug("TaskGroupCoordinator round cost: {}/ms", taskGroupCoordinatorRoundCost.getTime());
} finally {
registryClient.releaseLock(RegistryNodeType.MASTER_TASK_GROUP_COORDINATOR_LOCK.getRegistryPath());
}
@ -183,7 +183,7 @@ public class TaskGroupCoordinator extends BaseDaemonThread implements AutoClosea
}
minTaskGroupQueueId = taskGroupQueues.get(taskGroupQueues.size() - 1).getId();
}
log.info("Success amend TaskGroupQueue status cost: {}/ms", taskGroupCoordinatorRoundTimeCost.getTime());
log.debug("Success amend TaskGroupQueue status cost: {}/ms", taskGroupCoordinatorRoundTimeCost.getTime());
}
/**
@ -237,7 +237,7 @@ public class TaskGroupCoordinator extends BaseDaemonThread implements AutoClosea
}
minTaskGroupQueueId = taskGroupQueues.get(taskGroupQueues.size() - 1).getId();
}
log.info("Success deal with force start TaskGroupQueue cost: {}/ms",
log.debug("Success deal with force start TaskGroupQueue cost: {}/ms",
taskGroupCoordinatorRoundTimeCost.getTime());
}

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

@ -68,8 +68,4 @@ public class WorkflowCacheRepository implements IWorkflowRepository {
return ImmutableList.copyOf(workflowExecutionRunnableMap.values());
}
@Override
public void clear() {
workflowExecutionRunnableMap.clear();
}
}

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

@ -18,7 +18,8 @@
package org.apache.dolphinscheduler.server.master.engine;
import org.apache.dolphinscheduler.server.master.engine.command.CommandEngine;
import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecutorBootstrap;
import org.apache.dolphinscheduler.server.master.engine.executor.LogicTaskEngineDelegator;
import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueueLooper;
import lombok.extern.slf4j.Slf4j;
@ -36,31 +37,38 @@ public class WorkflowEngine implements AutoCloseable {
private WorkflowEventBusCoordinator workflowEventBusCoordinator;
@Autowired
private MasterTaskExecutorBootstrap masterTaskExecutorBootstrap;
private CommandEngine commandEngine;
@Autowired
private CommandEngine commandEngine;
private GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper;
@Autowired
private LogicTaskEngineDelegator logicTaskEngineDelegator;
public void start() {
taskGroupCoordinator.start();
masterTaskExecutorBootstrap.start();
workflowEventBusCoordinator.start();
commandEngine.start();
globalTaskDispatchWaitingQueueLooper.start();
logicTaskEngineDelegator.start();
log.info("WorkflowEngine started");
}
@Override
public void close() throws Exception {
try (
final CommandEngine commandEngine1 = commandEngine;
final WorkflowEventBusCoordinator workflowEventBusCoordinator1 = workflowEventBusCoordinator;
final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap;
final TaskGroupCoordinator taskGroupCoordinator1 = taskGroupCoordinator) {
final CommandEngine ignore1 = commandEngine;
final WorkflowEventBusCoordinator ignore2 = workflowEventBusCoordinator;
final GlobalTaskDispatchWaitingQueueLooper ignore3 =
globalTaskDispatchWaitingQueueLooper;
final TaskGroupCoordinator ignore4 = taskGroupCoordinator;
final LogicTaskEngineDelegator ignore5 = logicTaskEngineDelegator) {
// closed the resource
}
}

6
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorker.java

@ -78,7 +78,7 @@ public class WorkflowEventBusFireWorker {
if (CollectionUtils.isEmpty(workflowExecutionRunnables)) {
return;
}
for (IWorkflowExecutionRunnable workflowExecutionRunnable : workflowExecutionRunnables) {
for (final IWorkflowExecutionRunnable workflowExecutionRunnable : workflowExecutionRunnables) {
final Integer workflowInstanceId = workflowExecutionRunnable.getId();
final String workflowInstanceName = workflowExecutionRunnable.getName();
try {
@ -92,6 +92,10 @@ public class WorkflowEventBusFireWorker {
}
}
public int getRegisteredWorkflowExecuteRunnableSize() {
return registeredWorkflowExecuteRunnableMap.size();
}
private List<IWorkflowExecutionRunnable> getWaitingFireWorkflowExecutionRunnables() {
if (MapUtils.isEmpty(registeredWorkflowExecuteRunnableMap)) {
return Collections.emptyList();

9
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorkers.java

@ -30,6 +30,8 @@ import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import com.google.common.annotations.VisibleForTesting;
@Slf4j
@Component
public class WorkflowEventBusFireWorkers implements AutoCloseable {
@ -50,7 +52,7 @@ public class WorkflowEventBusFireWorkers implements AutoCloseable {
final int workflowEventBusFireThreadCount = masterConfig.getWorkflowEventBusFireThreadCount();
workflowEventBusFireThreadPool = Executors.newScheduledThreadPool(
workflowEventBusFireThreadCount,
ThreadUtils.newDaemonThreadFactory("DS-WorkflowEventBusFireWorker-%d"));
ThreadUtils.newDaemonThreadFactory("ds-workflow-eventbus-worker-%d"));
workflowEventBusFireWorkers = new WorkflowEventBusFireWorker[workflowEventBusFireThreadCount];
for (int i = 0; i < workflowEventBusFireThreadCount; i++) {
@ -73,6 +75,11 @@ public class WorkflowEventBusFireWorkers implements AutoCloseable {
return workflowEventBusFireWorkers[workerSlot];
}
@VisibleForTesting
public WorkflowEventBusFireWorker[] getWorkers() {
return workflowEventBusFireWorkers;
}
public int getWorkerSize() {
return masterConfig.getWorkflowEventBusFireThreadCount();
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/IdSlotBasedCommandFetcher.java

@ -64,7 +64,7 @@ public class IdSlotBasedCommandFetcher implements ICommandFetcher {
idSlotBasedFetchConfig.getIdStep(),
idSlotBasedFetchConfig.getFetchSize());
long cost = System.currentTimeMillis() - scheduleStartTime;
log.info("[Slot-{}/{}] Fetch {} commands in {}ms.", currentSlotIndex, totalSlot, commands.size(), cost);
log.debug("[Slot-{}/{}] Fetch {} commands in {}ms.", currentSlotIndex, totalSlot, commands.size(), cost);
WorkflowInstanceMetrics.recordCommandQueryTime(cost);
return commands;
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskReassignMasterHostException.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.exceptions;
public class TaskReassignMasterHostException extends RuntimeException {
public TaskReassignMasterHostException(String message) {
super(message);
}
public TaskReassignMasterHostException(String message, Throwable cause) {
super(message, cause);
}
}

78
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineDelegator.java

@ -0,0 +1,78 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import org.apache.dolphinscheduler.task.executor.TaskEngine;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class LogicTaskEngineDelegator implements AutoCloseable {
private final TaskEngine taskEngine;
private final LogicTaskExecutorFactory logicTaskExecutorFactory;
private final LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter;
public LogicTaskEngineDelegator(final LogicTaskEngineFactory logicTaskEngineFactory,
final LogicTaskExecutorFactory logicTaskExecutorFactory,
final LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter) {
this.logicTaskExecutorFactory = logicTaskExecutorFactory;
this.taskEngine = logicTaskEngineFactory.createTaskEngine();
this.logicTaskExecutorEventReporter = logicTaskExecutorEventReporter;
}
public void start() {
taskEngine.start();
logicTaskExecutorEventReporter.start();
log.info("LogicTaskEngineDelegator started");
}
public void dispatchLogicTask(final TaskExecutionContext taskExecutionContext) {
final ITaskExecutor taskExecutor = logicTaskExecutorFactory.createTaskExecutor(taskExecutionContext);
taskEngine.submitTask(taskExecutor);
}
public void killLogicTask(final int taskInstanceId) {
taskEngine.killTask(taskInstanceId);
}
public void pauseLogicTask(final int taskInstanceId) {
taskEngine.pauseTask(taskInstanceId);
}
public void ackLogicTaskExecutionEvent(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
logicTaskExecutorEventReporter.receiveTaskExecutorLifecycleEventACK(taskExecutorLifecycleEventAck);
}
@Override
public void close() {
try (
final TaskEngine ignore1 = taskEngine;
final LogicTaskExecutorLifecycleEventReporter ignore2 = logicTaskExecutorEventReporter) {
log.info("LogicTaskEngineDelegator closed");
}
}
}

52
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.engine.executor;
import org.apache.dolphinscheduler.task.executor.TaskEngine;
import org.apache.dolphinscheduler.task.executor.TaskEngineBuilder;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskEngineFactory {
private final LogicTaskExecutorRepository logicTaskExecutorRepository;
private final LogicTaskExecutorContainerProvider logicTaskExecutorContainerDelegator;
private final LogicTaskExecutorEventBusCoordinator logicTaskExecutorEventBusCoordinator;
public LogicTaskEngineFactory(final LogicTaskExecutorRepository logicTaskExecutorRepository,
final LogicTaskExecutorContainerProvider logicTaskExecutorContainerDelegator,
final LogicTaskExecutorEventBusCoordinator logicTaskExecutorEventBusCoordinator) {
this.logicTaskExecutorRepository = logicTaskExecutorRepository;
this.logicTaskExecutorContainerDelegator = logicTaskExecutorContainerDelegator;
this.logicTaskExecutorEventBusCoordinator = logicTaskExecutorEventBusCoordinator;
}
public TaskEngine createTaskEngine() {
final TaskEngineBuilder taskEngineBuilder = TaskEngineBuilder.builder()
.engineName("LogicTaskEngine")
.taskExecutorRepository(logicTaskExecutorRepository)
.taskExecutorContainerDelegator(logicTaskExecutorContainerDelegator)
.taskExecutorEventBusCoordinator(logicTaskExecutorEventBusCoordinator)
.build();
return new TaskEngine(taskEngineBuilder);
}
}

88
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutor.java

@ -0,0 +1,88 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.LogicTaskPluginFactoryBuilder;
import org.apache.dolphinscheduler.task.executor.AbstractTaskExecutor;
import org.apache.dolphinscheduler.task.executor.TaskExecutorState;
import org.apache.dolphinscheduler.task.executor.TaskExecutorStateMappings;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class LogicTaskExecutor extends AbstractTaskExecutor {
private ILogicTask<? extends AbstractParameters> logicTask;
private final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
public LogicTaskExecutor(final LogicTaskExecutorBuilder logicTaskExecutorBuilder) {
super(logicTaskExecutorBuilder.getTaskExecutionContext(), logicTaskExecutorBuilder.getTaskExecutorEventBus());
this.logicTaskPluginFactoryBuilder = logicTaskExecutorBuilder.getLogicTaskPluginFactoryBuilder();
}
@Override
protected TaskExecutorState doTrackTaskPluginStatus() {
return TaskExecutorStateMappings.mapState(logicTask.getTaskExecutionState());
}
@SneakyThrows
@Override
protected void initializeTaskPlugin() {
logicTask = logicTaskPluginFactoryBuilder
.createILogicTaskPluginFactory(taskExecutionContext.getTaskType())
.createLogicTask(this);
log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType());
}
@SneakyThrows
@Override
protected void doTriggerTaskPlugin() {
logicTask.start();
}
@SneakyThrows
@Override
public void pause() {
// todo: judge the status, whether the task is running, we should support to pause the task which is not running
// if the status is initialized, then we can directly change to paused
if (logicTask != null) {
logicTask.pause();
}
}
@SneakyThrows
@Override
public void kill() {
if (logicTask != null) {
logicTask.kill();
}
}
@Override
public String toString() {
return "LogicTaskExecutor{" +
"id=" + taskExecutionContext.getTaskInstanceId() +
", name=" + taskExecutionContext.getTaskName() +
", stat=" + taskExecutorState.get() +
'}';
}
}

31
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java

@ -15,37 +15,26 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.extract.master.transportor;
package org.apache.dolphinscheduler.server.master.engine.executor;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.LogicTaskPluginFactoryBuilder;
import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorEventBus;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
@Data
@Builder
@NoArgsConstructor
@AllArgsConstructor
public class TaskExecutionFailedEvent implements ITaskExecutionEvent {
private int taskInstanceId;
private int workflowInstanceId;
private String taskInstanceHost;
private String workflowInstanceHost;
private String appIds;
public class LogicTaskExecutorBuilder {
private long endTime;
private TaskExecutionContext taskExecutionContext;
private long eventCreateTime;
@Builder.Default
private TaskExecutorEventBus taskExecutorEventBus = new TaskExecutorEventBus();
private long eventSendTime;
private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
@Override
public TaskInstanceExecutionEventType getEventType() {
return TaskInstanceExecutionEventType.FAILED;
}
}

45
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerProvider.java

@ -0,0 +1,45 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer;
import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerProvider;
import org.apache.dolphinscheduler.task.executor.container.SharedThreadTaskExecutorContainer;
import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerConfig;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskExecutorContainerProvider implements ITaskExecutorContainerProvider {
private final ITaskExecutorContainer taskExecutorContainer;
public LogicTaskExecutorContainerProvider(final MasterConfig masterConfig) {
final TaskExecutorContainerConfig containerConfig = TaskExecutorContainerConfig.builder()
.taskExecutorThreadPoolSize(masterConfig.getLogicTaskConfig().getTaskExecutorThreadCount())
.containerName("shared-task-executor-container")
.build();
this.taskExecutorContainer = new SharedThreadTaskExecutorContainer(containerConfig);
}
@Override
public ITaskExecutorContainer getExecutorContainer() {
return taskExecutorContainer;
}
}

28
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java

@ -15,32 +15,18 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
package org.apache.dolphinscheduler.server.master.engine.executor;
import java.util.concurrent.DelayQueue;
import javax.annotation.Nullable;
import lombok.NonNull;
import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorEventBusCoordinator;
import org.springframework.stereotype.Component;
@Component
public class AsyncMasterTaskDelayQueue {
private final DelayQueue<AsyncTaskExecutionContext> asyncTaskCheckDelayQueue = new DelayQueue<>();
public void addAsyncTask(@NonNull AsyncTaskExecutionContext asyncTaskExecutionContext) {
asyncTaskExecutionContext.refreshStartTime();
asyncTaskCheckDelayQueue.add(asyncTaskExecutionContext);
}
public @Nullable AsyncTaskExecutionContext pollAsyncTask() throws InterruptedException {
return asyncTaskCheckDelayQueue.take();
}
public class LogicTaskExecutorEventBusCoordinator extends TaskExecutorEventBusCoordinator {
public int getAsyncTaskRunningNum() {
return asyncTaskCheckDelayQueue.size();
public LogicTaskExecutorEventBusCoordinator(final LogicTaskExecutorRepository logicTaskExecutorRepository,
final LogicTaskExecutorLifecycleEventListener logicTaskExecutorLifecycleEventListener) {
super("LogicTaskExecutorEventBusCoordinator", logicTaskExecutorRepository);
registerTaskExecutorLifecycleEventListener(logicTaskExecutorLifecycleEventListener);
}
}

27
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventRemoteReporterClient.java

@ -0,0 +1,27 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.extract.master.TaskExecutorEventRemoteReporterClient;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskExecutorEventRemoteReporterClient extends TaskExecutorEventRemoteReporterClient {
}

55
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorFactory.java

@ -0,0 +1,55 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.LogicTaskPluginFactoryBuilder;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import org.apache.dolphinscheduler.task.executor.ITaskExecutorFactory;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class LogicTaskExecutorFactory implements ITaskExecutorFactory {
private final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
public LogicTaskExecutorFactory(final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder) {
this.logicTaskPluginFactoryBuilder = logicTaskPluginFactoryBuilder;
}
@Override
public ITaskExecutor createTaskExecutor(final TaskExecutionContext taskExecutionContext) {
assemblyTaskLogPath(taskExecutionContext);
final LogicTaskExecutorBuilder logicTaskExecutorBuilder = LogicTaskExecutorBuilder.builder()
.taskExecutionContext(taskExecutionContext)
.logicTaskPluginFactoryBuilder(logicTaskPluginFactoryBuilder)
.build();
return new LogicTaskExecutor(logicTaskExecutorBuilder);
}
private void assemblyTaskLogPath(final TaskExecutionContext taskExecutionContext) {
taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext));
}
}

36
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.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.server.master.engine.executor;
import org.apache.dolphinscheduler.task.executor.listener.TaskExecutorLifecycleEventListener;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskExecutorLifecycleEventListener extends TaskExecutorLifecycleEventListener {
public LogicTaskExecutorLifecycleEventListener(
final LogicTaskExecutorContainerProvider logicTaskExecutorContainerDelegator,
final LogicTaskExecutorRepository logicTaskExecutorRepository,
final LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter) {
super(
logicTaskExecutorContainerDelegator,
logicTaskExecutorRepository,
logicTaskExecutorEventReporter);
}
}

31
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventReporter.java

@ -0,0 +1,31 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorLifecycleEventRemoteReporter;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskExecutorLifecycleEventReporter extends TaskExecutorLifecycleEventRemoteReporter {
public LogicTaskExecutorLifecycleEventReporter(
final LogicTaskExecutorEventRemoteReporterClient logicTaskExecutorEventRemoteReporterClient) {
super("LogicTaskExecutorLifecycleEventReporter", logicTaskExecutorEventRemoteReporterClient);
}
}

27
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorRepository.java

@ -0,0 +1,27 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.task.executor.TaskExecutorRepository;
import org.springframework.stereotype.Component;
@Component
public class LogicTaskExecutorRepository extends TaskExecutorRepository {
}

74
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/AbstractLogicTask.java

@ -0,0 +1,74 @@
/*
* 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.executor.plugin;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import lombok.extern.slf4j.Slf4j;
import com.google.common.base.Preconditions;
@Slf4j
public abstract class AbstractLogicTask<T extends AbstractParameters> implements ILogicTask<T> {
protected final TaskExecutionContext taskExecutionContext;
protected final T taskParameters;
protected TaskExecutionStatus taskExecutionStatus;
public AbstractLogicTask(final TaskExecutionContext taskExecutionContext) {
this.taskExecutionContext = taskExecutionContext;
this.taskParameters = getTaskParameterDeserializer().deserialize(taskExecutionContext.getTaskParams());
Preconditions.checkNotNull(taskParameters,
"Deserialize task parameters: " + taskExecutionContext.getTaskParams());
log.info("Success initialize parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters));
}
@Override
public TaskExecutionStatus getTaskExecutionState() {
return taskExecutionStatus;
}
protected boolean isRunning() {
return taskExecutionStatus == TaskExecutionStatus.RUNNING_EXECUTION;
}
protected void onTaskRunning() {
taskExecutionStatus = TaskExecutionStatus.RUNNING_EXECUTION;
}
protected void onTaskSuccess() {
taskExecutionStatus = TaskExecutionStatus.SUCCESS;
}
protected void onTaskFailed() {
taskExecutionStatus = TaskExecutionStatus.FAILURE;
}
protected void onTaskKilled() {
taskExecutionStatus = TaskExecutionStatus.KILL;
}
protected void onTaskPaused() {
taskExecutionStatus = TaskExecutionStatus.PAUSE;
}
}

12
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTask.java

@ -15,20 +15,22 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
public interface ILogicTask {
public interface ILogicTask<T extends AbstractParameters> {
void start() throws MasterTaskExecuteException;
void pause() throws MasterTaskExecuteException;
void kill() throws MasterTaskExecuteException;
AbstractParameters getTaskParameters();
TaskExecutionStatus getTaskExecutionState();
TaskExecutionContext getTaskExecutionContext();
ITaskParameterDeserializer<T> getTaskParameterDeserializer();
}

9
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTaskPluginFactory.java

@ -15,14 +15,15 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
public interface ILogicTaskPluginFactory<T extends ILogicTask> {
public interface ILogicTaskPluginFactory<T extends ILogicTask<? extends AbstractParameters>> {
T createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException;
T createLogicTask(final ITaskExecutor taskExecutor) throws LogicTaskInitializeException;
String getTaskType();

6
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java

@ -15,10 +15,10 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.worker.runner.operator;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin;
public interface ITaskInstanceOperationFunction<X, Y> {
public interface ITaskParameterDeserializer<T> {
Y operate(X x);
T deserialize(String taskParamsJson);
}

14
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/LogicTaskPluginFactoryBuilder.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/LogicTaskPluginFactoryBuilder.java

@ -15,8 +15,9 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskFactoryNotFoundException;
import java.util.List;
@ -29,19 +30,20 @@ import org.springframework.stereotype.Component;
@Slf4j
@Component
@SuppressWarnings("unchecked")
public class LogicTaskPluginFactoryBuilder {
private final Map<String, ILogicTaskPluginFactory> logicTaskPluginFactoryMap = new ConcurrentHashMap<>();
private final Map<String, ILogicTaskPluginFactory<? extends ILogicTask<? extends AbstractParameters>>> logicTaskPluginFactoryMap =
new ConcurrentHashMap<>();
public LogicTaskPluginFactoryBuilder(List<ILogicTaskPluginFactory> logicTaskPluginFactories) {
public LogicTaskPluginFactoryBuilder(List<ILogicTaskPluginFactory<? extends ILogicTask<? extends AbstractParameters>>> logicTaskPluginFactories) {
logicTaskPluginFactories.forEach(
logicTaskPluginFactory -> logicTaskPluginFactoryMap.put(logicTaskPluginFactory.getTaskType(),
logicTaskPluginFactory));
}
public ILogicTaskPluginFactory createILogicTaskPluginFactory(String taskType) throws LogicTaskFactoryNotFoundException {
ILogicTaskPluginFactory logicTaskPluginFactory = logicTaskPluginFactoryMap.get(taskType);
public ILogicTaskPluginFactory<? extends ILogicTask<? extends AbstractParameters>> createILogicTaskPluginFactory(String taskType) throws LogicTaskFactoryNotFoundException {
ILogicTaskPluginFactory<? extends ILogicTask<? extends AbstractParameters>> logicTaskPluginFactory =
logicTaskPluginFactoryMap.get(taskType);
if (logicTaskPluginFactory == null) {
throw new LogicTaskFactoryNotFoundException("Cannot find the logic task factory: " + taskType);
}

47
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTask.java

@ -15,19 +15,20 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.condition;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.condition;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import java.util.List;
import java.util.Map;
@ -40,28 +41,35 @@ import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j
public class ConditionLogicTask extends BaseSyncLogicTask<ConditionsParameters> {
public static final String TASK_TYPE = "CONDITIONS";
public class ConditionLogicTask extends AbstractLogicTask<ConditionsParameters> {
private final TaskInstanceDao taskInstanceDao;
private final TaskInstance taskInstance;
public ConditionLogicTask(IWorkflowExecutionRunnable workflowExecutionRunnable,
TaskExecutionContext taskExecutionContext,
TaskInstanceDao taskInstanceDao) {
super(workflowExecutionRunnable, taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<ConditionsParameters>() {
}));
super(taskExecutionContext);
this.taskInstance = workflowExecutionRunnable
.getWorkflowExecuteContext()
.getWorkflowExecutionGraph()
.getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId())
.getTaskInstance();
this.taskInstanceDao = taskInstanceDao;
onTaskRunning();
}
@Override
public void handle() {
public void start() {
DependResult conditionResult = calculateConditionResult();
log.info("The condition result is {}", conditionResult);
taskParameters.getConditionResult().setConditionSuccess(conditionResult == DependResult.SUCCESS);
taskInstance.setTaskParams(JSONUtils.toJsonString(taskParameters));
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
onTaskSuccess();
}
private DependResult calculateConditionResult() {
@ -100,4 +108,21 @@ public class ConditionLogicTask extends BaseSyncLogicTask<ConditionsParameters>
log.info("The dependent result will be: {}", dependResult);
return dependResult;
}
@Override
public void pause() throws MasterTaskExecuteException {
log.info("The ConditionTask does not support pause operation");
}
@Override
public void kill() throws MasterTaskExecuteException {
log.info("The ConditionTask does not support kill operation");
}
@Override
public ITaskParameterDeserializer<ConditionsParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<ConditionsParameters>() {
});
}
}

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTaskPluginFactory.java

@ -15,14 +15,15 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.condition;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.condition;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import lombok.extern.slf4j.Slf4j;
@ -40,8 +41,9 @@ public class ConditionLogicTaskPluginFactory implements ILogicTaskPluginFactory<
private IWorkflowRepository workflowExecutionRunnableMemoryRepository;
@Override
public ConditionLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) {
IWorkflowExecutionRunnable workflowExecutionRunnable =
public ConditionLogicTask createLogicTask(final ITaskExecutor taskExecutor) {
final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext();
final IWorkflowExecutionRunnable workflowExecutionRunnable =
workflowExecutionRunnableMemoryRepository.get(taskExecutionContext.getWorkflowInstanceId());
return new ConditionLogicTask(workflowExecutionRunnable, taskExecutionContext, taskInstanceDao);
}

68
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTask.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dependent;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dependent;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.repository.ProjectDao;
@ -24,30 +24,23 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j
public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters> {
public class DependentLogicTask extends AbstractLogicTask<DependentParameters> {
public static final String TASK_TYPE = "DEPENDENT";
private final TaskExecutionContext taskExecutionContext;
private final ProjectDao projectDao;
private final WorkflowDefinitionDao workflowDefinitionDao;
private final TaskDefinitionDao taskDefinitionDao;
private final TaskInstanceDao taskInstanceDao;
private final WorkflowInstanceDao workflowInstanceDao;
private final IWorkflowExecutionRunnable workflowExecutionRunnable;
private DependentAsyncTaskExecuteFunction dependentAsyncTaskExecuteFunction;
private final DependentTaskTracker dependentTaskTracker;
public DependentLogicTask(TaskExecutionContext taskExecutionContext,
ProjectDao projectDao,
@ -56,38 +49,49 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
TaskInstanceDao taskInstanceDao,
WorkflowInstanceDao workflowInstanceDao,
IWorkflowExecutionRunnable workflowExecutionRunnable) {
super(taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<DependentParameters>() {
}));
this.projectDao = projectDao;
this.workflowDefinitionDao = workflowDefinitionDao;
this.taskDefinitionDao = taskDefinitionDao;
this.taskInstanceDao = taskInstanceDao;
this.workflowInstanceDao = workflowInstanceDao;
this.workflowExecutionRunnable = workflowExecutionRunnable;
}
@Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
dependentAsyncTaskExecuteFunction = new DependentAsyncTaskExecuteFunction(taskExecutionContext,
super(taskExecutionContext);
this.taskExecutionContext = taskExecutionContext;
this.dependentTaskTracker = new DependentTaskTracker(
taskExecutionContext,
taskParameters,
projectDao,
workflowDefinitionDao,
taskDefinitionDao,
taskInstanceDao,
workflowInstanceDao);
return dependentAsyncTaskExecuteFunction;
onTaskRunning();
}
@Override
public void start() throws MasterTaskExecuteException {
log.info("Dependent task: {} started", taskExecutionContext.getTaskName());
}
@Override
public TaskExecutionStatus getTaskExecutionState() {
if (isRunning()) {
taskExecutionStatus = dependentTaskTracker.getDependentTaskStatus();
return taskExecutionStatus;
}
return taskExecutionStatus;
}
@Override
public void pause() throws MasterTaskExecuteException {
// todo: support pause
onTaskPaused();
log.info("Pause task : {} success", taskExecutionContext.getTaskName());
}
@Override
public void kill() throws MasterTaskExecuteException {
// todo: support kill
onTaskKilled();
log.info("Kill task : {} success", taskExecutionContext.getTaskName());
}
@Override
public ITaskParameterDeserializer<DependentParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<DependentParameters>() {
});
}
}

11
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTaskPluginFactory.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dependent;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dependent;
import org.apache.dolphinscheduler.dao.repository.ProjectDao;
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao;
@ -23,10 +23,12 @@ import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.DependentLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import lombok.extern.slf4j.Slf4j;
@ -52,7 +54,8 @@ public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory<
private IWorkflowRepository IWorkflowRepository;
@Override
public DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
public DependentLogicTask createLogicTask(final ITaskExecutor taskExecutor) throws LogicTaskInitializeException {
final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext();
final int workflowInstanceId = taskExecutionContext.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflowExecutionRunnable = IWorkflowRepository.get(workflowInstanceId);
if (workflowExecutionRunnable == null) {
@ -70,6 +73,6 @@ public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory<
@Override
public String getTaskType() {
return DependentLogicTask.TASK_TYPE;
return DependentLogicTaskChannelFactory.NAME;
}
}

25
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentAsyncTaskExecuteFunction.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentTaskTracker.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dependent;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dependent;
import static org.apache.dolphinscheduler.common.constants.Constants.DEPENDENT_SPLIT;
@ -33,15 +33,14 @@ import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.utils.DependentExecute;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
@ -56,9 +55,7 @@ import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction {
private static final Duration DEFAULT_STATE_CHECK_INTERVAL = Duration.ofSeconds(10);
public class DependentTaskTracker {
private final TaskExecutionContext taskExecutionContext;
private final DependentParameters dependentParameters;
@ -73,7 +70,7 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
private final Map<String, DependResult> dependResultMap;
private final Map<String, Property> dependVarPoolPropertyMap;
public DependentAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext,
public DependentTaskTracker(TaskExecutionContext taskExecutionContext,
DependentParameters dependentParameters,
ProjectDao projectDao,
WorkflowDefinitionDao workflowDefinitionDao,
@ -95,8 +92,7 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
this.dependVarPoolPropertyMap = new HashMap<>();
}
@Override
public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
public @NonNull TaskExecutionStatus getDependentTaskStatus() {
if (isAllDependentTaskFinished()) {
log.info("All dependent task finished, will calculate the dependent result");
DependResult dependResult = calculateDependResult();
@ -104,12 +100,12 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
if (dependResult == DependResult.SUCCESS) {
dependentParameters.setVarPool(JSONUtils.toJsonString(dependVarPoolPropertyMap.values()));
log.info("Set dependentParameters varPool: {}", dependentParameters.getVarPool());
return AsyncTaskExecutionStatus.SUCCESS;
return TaskExecutionStatus.SUCCESS;
} else {
return AsyncTaskExecutionStatus.FAILED;
return TaskExecutionStatus.FAILURE;
}
}
return AsyncTaskExecutionStatus.RUNNING;
return TaskExecutionStatus.RUNNING_EXECUTION;
}
private Date calculateDependentDate() {
@ -239,9 +235,4 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
return isAllDependentTaskFinished;
}
@Override
public @NonNull Duration getAsyncTaskStateCheckInterval() {
return dependentParameters.getDependence().getCheckInterval() == null ? DEFAULT_STATE_CHECK_INTERVAL
: Duration.ofSeconds(dependentParameters.getDependence().getCheckInterval());
}
}

24
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunction.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicAsyncTaskExecuteFunction.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic;
import static org.apache.dolphinscheduler.common.constants.CommandKeyConstants.CMD_DYNAMIC_START_PARAMS;
@ -27,8 +27,8 @@ import org.apache.dolphinscheduler.dao.mapper.CommandMapper;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.DataType;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService;
import java.time.Duration;
@ -41,7 +41,7 @@ import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction {
public class DynamicAsyncTaskExecuteFunction {
private static final Duration TASK_EXECUTE_STATE_CHECK_INTERVAL = Duration.ofSeconds(10);
@ -75,8 +75,7 @@ public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction
this.subWorkflowService = subWorkflowService;
}
@Override
public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
public @NonNull TaskExecutionStatus getAsyncTaskExecutionStatus() {
List<WorkflowInstance> allSubWorkflowInstance = getAllSubProcessInstance();
int totalSubProcessInstanceCount = allSubWorkflowInstance.size();
@ -90,16 +89,16 @@ public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction
if (successCount == totalSubProcessInstanceCount) {
log.info("all sub process instance success");
setOutputParameters();
return AsyncTaskExecutionStatus.SUCCESS;
return TaskExecutionStatus.SUCCESS;
} else {
int failedCount = totalSubProcessInstanceCount - successCount;
log.info("failed sub process instance count: {}", failedCount);
return AsyncTaskExecutionStatus.FAILED;
return TaskExecutionStatus.FAILURE;
}
}
if (logicTask.isCancel()) {
return AsyncTaskExecutionStatus.FAILED;
return TaskExecutionStatus.FAILURE;
}
int runningCount = subWorkflowService.filterRunningProcessInstances(allSubWorkflowInstance).size();
@ -109,7 +108,7 @@ public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction
startSubProcessInstances(allSubWorkflowInstance, startCount);
}
// query the status of sub workflow instance
return AsyncTaskExecutionStatus.RUNNING;
return TaskExecutionStatus.RUNNING_EXECUTION;
}
private void setOutputParameters() {
@ -143,7 +142,7 @@ public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction
List<Property> taskPropertyList = new ArrayList<>(JSONUtils.toList(taskInstance.getVarPool(), Property.class));
taskPropertyList.add(property);
logicTask.getTaskParameters().setVarPool(JSONUtils.toJsonString(taskPropertyList));
// logicTask.getTaskParameters().setVarPool(JSONUtils.toJsonString(taskPropertyList));
log.info("set property: {}", property);
}
@ -171,9 +170,4 @@ public class DynamicAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction
return subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode());
}
@Override
public @NonNull Duration getAsyncTaskStateCheckInterval() {
return TASK_EXECUTE_STATE_CHECK_INTERVAL;
}
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtils.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicCommandUtils.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic;
import org.apache.dolphinscheduler.common.constants.CommandKeyConstants;
import org.apache.dolphinscheduler.common.enums.CommandType;

75
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTask.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic;
import org.apache.dolphinscheduler.common.constants.CommandKeyConstants;
import org.apache.dolphinscheduler.common.enums.Flag;
@ -35,13 +35,14 @@ import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
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.parameters.DynamicParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService;
@ -61,7 +62,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.Lists;
@Slf4j
public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
public class DynamicLogicTask extends AbstractLogicTask<DynamicParameters> {
public static final String TASK_TYPE = "DYNAMIC";
private final WorkflowInstanceDao workflowInstanceDao;
@ -78,6 +79,8 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
private TaskInstance taskInstance;
private final TaskExecutionContext taskExecutionContext;
private boolean haveBeenCanceled = false;
public DynamicLogicTask(TaskExecutionContext taskExecutionContext,
@ -87,9 +90,8 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
ProcessService processService,
WorkflowDefinitionMapper workflowDefinitionMapper,
CommandMapper commandMapper) {
super(taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<DynamicParameters>() {
}));
super(taskExecutionContext);
this.taskExecutionContext = taskExecutionContext;
this.workflowInstanceDao = workflowInstanceDao;
this.subWorkflowService = subWorkflowService;
this.processService = processService;
@ -100,28 +102,27 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
this.taskInstance = taskInstanceDao.queryById(taskExecutionContext.getTaskInstanceId());
}
@Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException {
List<Map<String, String>> parameterGroup = generateParameterGroup();
if (parameterGroup.size() > taskParameters.getMaxNumOfSubWorkflowInstances()) {
log.warn("the number of sub process instances [{}] exceeds the maximum limit [{}]", parameterGroup.size(),
taskParameters.getMaxNumOfSubWorkflowInstances());
parameterGroup = parameterGroup.subList(0, taskParameters.getMaxNumOfSubWorkflowInstances());
}
// if already exists sub process instance, do not generate again
List<WorkflowInstance> existsSubWorkflowInstanceList =
subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode());
if (CollectionUtils.isEmpty(existsSubWorkflowInstanceList)) {
generateSubWorkflowInstance(parameterGroup);
} else {
resetProcessInstanceStatus(existsSubWorkflowInstanceList);
}
return new DynamicAsyncTaskExecuteFunction(taskExecutionContext, workflowInstance, taskInstance, this,
commandMapper,
subWorkflowService, taskParameters.getDegreeOfParallelism());
}
// public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException {
// List<Map<String, String>> parameterGroup = generateParameterGroup();
//
// if (parameterGroup.size() > dynamicParameters.getMaxNumOfSubWorkflowInstances()) {
// log.warn("the number of sub process instances [{}] exceeds the maximum limit [{}]", parameterGroup.size(),
// dynamicParameters.getMaxNumOfSubWorkflowInstances());
// parameterGroup = parameterGroup.subList(0, dynamicParameters.getMaxNumOfSubWorkflowInstances());
// }
//
// // if already exists sub process instance, do not generate again
// List<WorkflowInstance> existsSubWorkflowInstanceList =
// subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode());
// if (CollectionUtils.isEmpty(existsSubWorkflowInstanceList)) {
// generateSubWorkflowInstance(parameterGroup);
// } else {
// resetProcessInstanceStatus(existsSubWorkflowInstanceList);
// }
// return new DynamicAsyncTaskExecuteFunction(taskExecutionContext, workflowInstance, taskInstance, this,
// commandMapper,
// subWorkflowService, dynamicParameters.getDegreeOfParallelism());
// }
public void resetProcessInstanceStatus(List<WorkflowInstance> existsSubWorkflowInstanceList) {
switch (workflowInstance.getCommandType()) {
@ -250,6 +251,16 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
return dynamicInputParameters;
}
@Override
public void start() throws MasterTaskExecuteException {
// todo:
}
@Override
public TaskExecutionStatus getTaskExecutionState() {
return taskExecutionContext.getCurrentExecutionStatus();
}
@Override
public void pause() throws MasterTaskExecuteException {
// todo: support pause
@ -264,6 +275,12 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
}
}
@Override
public ITaskParameterDeserializer<DynamicParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<DynamicParameters>() {
});
}
private void doKillSubWorkflowInstances() throws MasterTaskExecuteException {
List<WorkflowInstance> existsSubWorkflowInstanceList =
subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode());

16
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTaskPluginFactory.java

@ -15,16 +15,17 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic;
import org.apache.dolphinscheduler.dao.mapper.CommandMapper;
import org.apache.dolphinscheduler.dao.mapper.WorkflowDefinitionMapper;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import lombok.extern.slf4j.Slf4j;
@ -54,10 +55,15 @@ public class DynamicLogicTaskPluginFactory implements ILogicTaskPluginFactory<Dy
SubWorkflowService subWorkflowService;
@Override
public DynamicLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) {
return new DynamicLogicTask(taskExecutionContext, workflowInstanceDao, taskInstanceDao, subWorkflowService,
public DynamicLogicTask createLogicTask(final ITaskExecutor taskExecutor) {
final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext();
return new DynamicLogicTask(taskExecutionContext,
workflowInstanceDao,
taskInstanceDao,
subWorkflowService,
processService,
processDefineMapper, commandMapper);
processDefineMapper,
commandMapper);
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicOutput.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicOutput.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic;
import java.util.Map;

55
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/fake/LogicFakeTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTask.java

@ -15,19 +15,20 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.fake;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.fake;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.LogicFakeTaskParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask;
import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.annotations.VisibleForTesting;
/**
@ -36,49 +37,65 @@ import com.google.common.annotations.VisibleForTesting;
*/
@Slf4j
@VisibleForTesting
public class LogicFakeTask extends BaseSyncLogicTask<LogicFakeTaskParameters> {
private volatile boolean killFlag;
public class LogicFakeTask extends AbstractLogicTask<LogicFakeTaskParameters> {
private Process process;
public LogicFakeTask(final IWorkflowExecutionRunnable workflowExecutionRunnable,
final TaskExecutionContext taskExecutionContext) {
super(workflowExecutionRunnable, taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), LogicFakeTaskParameters.class));
public LogicFakeTask(final TaskExecutionContext taskExecutionContext) {
super(taskExecutionContext);
onTaskRunning();
}
@Override
public void handle() throws MasterTaskExecuteException {
public void start() throws MasterTaskExecuteException {
try {
log.info("Begin to execute LogicFakeTask: {}", taskExecutionContext.getTaskName());
final String shellScript = ParameterUtils.convertParameterPlaceholders(
taskParameters.getShellScript(),
ParameterUtils.convert(taskExecutionContext.getPrepareParamsMap()));
final String[] cmd = {"/bin/sh", "-c", shellScript};
process = Runtime.getRuntime().exec(cmd);
int exitCode = process.waitFor();
if (killFlag) {
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.KILL);
if (taskExecutionStatus != TaskExecutionStatus.RUNNING_EXECUTION) {
// The task has been killed
return;
}
if (exitCode == 0) {
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
log.info("LogicFakeTask: {} execute success with exit code: {}",
taskExecutionContext.getTaskName(),
exitCode);
onTaskSuccess();
} else {
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE);
log.info("LogicFakeTask: {} execute failed with exit code: {}",
taskExecutionContext.getTaskName(),
exitCode);
onTaskFailed();
}
} catch (Exception ex) {
throw new MasterTaskExecuteException("FakeTask execute failed", ex);
}
}
@Override
public void pause() throws MasterTaskExecuteException {
log.info("LogicFakeTask: {} doesn't support pause", taskExecutionContext.getTaskName());
}
@Override
public void kill() throws MasterTaskExecuteException {
log.info("kill task : {}", taskExecutionContext.getTaskName());
if (process != null && process.isAlive()) {
killFlag = true;
process.destroy();
log.info("kill task : {} succeed", taskExecutionContext.getTaskName());
// todo: use shell script to kill the process?
onTaskKilled();
process.destroyForcibly();
log.info("Kill LogicFakeTask: {} succeed", taskExecutionContext.getTaskName());
}
}
@Override
public ITaskParameterDeserializer<LogicFakeTaskParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<LogicFakeTaskParameters>() {
});
}
}

25
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/fake/LogicFakeTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTaskPluginFactory.java

@ -15,15 +15,13 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.fake;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.fake;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.LogicFakeTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import com.google.common.annotations.VisibleForTesting;
@ -32,18 +30,15 @@ import com.google.common.annotations.VisibleForTesting;
@VisibleForTesting
public class LogicFakeTaskPluginFactory implements ILogicTaskPluginFactory<LogicFakeTask> {
@Autowired
private IWorkflowRepository IWorkflowRepository;
private final IWorkflowRepository workflowRepository;
@Override
public LogicFakeTask createLogicTask(final TaskExecutionContext taskExecutionContext) {
final IWorkflowExecutionRunnable workflowExecutionRunnable =
IWorkflowRepository.get(taskExecutionContext.getWorkflowInstanceId());
if (workflowExecutionRunnable == null) {
throw new IllegalStateException(
"Cannot find the WorkflowExecuteRunnable: " + taskExecutionContext.getWorkflowInstanceId());
public LogicFakeTaskPluginFactory(final IWorkflowRepository workflowRepository) {
this.workflowRepository = workflowRepository;
}
return new LogicFakeTask(workflowExecutionRunnable, taskExecutionContext);
@Override
public LogicFakeTask createLogicTask(final ITaskExecutor taskExecutor) {
return new LogicFakeTask(taskExecutor.getTaskExecutionContext());
}
@Override

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowControlClient.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
@ -31,10 +31,10 @@ import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowI
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow.trigger.SubWorkflowManualTrigger;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverFailureTaskTrigger;
import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverSuspendTaskTrigger;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.trigger.SubWorkflowManualTrigger;
import lombok.extern.slf4j.Slf4j;

49
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTask.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
@ -34,12 +34,14 @@ import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowI
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubWorkflowParameters;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import org.apache.dolphinscheduler.task.executor.events.TaskExecutorRuntimeContextChangedLifecycleEvent;
import lombok.extern.slf4j.Slf4j;
@ -48,7 +50,7 @@ import org.springframework.context.ApplicationContext;
import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j
public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubWorkflowParameters> {
public class SubWorkflowLogicTask extends AbstractLogicTask<SubWorkflowParameters> {
private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
@ -56,35 +58,48 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubWorkflowParamete
private final ApplicationContext applicationContext;
private SubWorkflowTracker subWorkflowTracker;
private ITaskExecutor taskExecutor;
public SubWorkflowLogicTask(final TaskExecutionContext taskExecutionContext,
final IWorkflowExecutionRunnable workflowExecutionRunnable,
final ITaskExecutor taskExecutor,
final ApplicationContext applicationContext) {
super(taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SubWorkflowParameters>() {
}));
super(taskExecutionContext);
this.taskExecutor = taskExecutor;
this.workflowExecutionRunnable = workflowExecutionRunnable;
this.applicationContext = applicationContext;
this.subWorkflowLogicTaskRuntimeContext = JSONUtils.parseObject(
taskExecutionContext.getAppIds(),
SubWorkflowLogicTaskRuntimeContext.class);
onTaskRunning();
}
@Override
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
public void start() throws MasterTaskExecuteException {
subWorkflowLogicTaskRuntimeContext = initializeSubWorkflowInstance();
upsertSubWorkflowRelation();
taskExecutionContext.setAppIds(JSONUtils.toJsonString(subWorkflowLogicTaskRuntimeContext));
applicationContext
.getBean(LogicTaskInstanceExecutionEventSenderManager.class)
.runningEventSender()
.sendMessage(taskExecutionContext);
taskExecutor.getTaskExecutorEventBus()
.publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(taskExecutor));
return new SubWorkflowAsyncTaskExecuteFunction(
subWorkflowTracker = new SubWorkflowTracker(
subWorkflowLogicTaskRuntimeContext,
applicationContext.getBean(WorkflowInstanceDao.class));
}
@Override
public TaskExecutionStatus getTaskExecutionState() {
if (subWorkflowTracker == null) {
// The sub workflow has not been started
return taskExecutionStatus;
}
taskExecutionStatus = subWorkflowTracker.getSubWorkflowState();
return taskExecutionStatus;
}
@Override
public void pause() throws MasterTaskExecuteException {
if (subWorkflowLogicTaskRuntimeContext == null) {
@ -120,6 +135,12 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubWorkflowParamete
}
}
@Override
public ITaskParameterDeserializer<SubWorkflowParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<SubWorkflowParameters>() {
});
}
private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowInstance() {
// todo: doFailover if the runtime context is not null and task is generated by failover

34
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskPluginFactory.java

@ -15,18 +15,17 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.ApplicationContext;
import org.springframework.stereotype.Component;
@ -34,21 +33,26 @@ import org.springframework.stereotype.Component;
@Component
public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactory<SubWorkflowLogicTask> {
@Autowired
private ApplicationContext applicationContext;
private final ApplicationContext applicationContext;
@Autowired
private IWorkflowRepository IWorkflowRepository;
private final IWorkflowRepository workflowRepository;
public SubWorkflowLogicTaskPluginFactory(final ApplicationContext applicationContext,
final IWorkflowRepository workflowRepository) {
this.applicationContext = applicationContext;
this.workflowRepository = workflowRepository;
}
@Override
public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
public SubWorkflowLogicTask createLogicTask(final ITaskExecutor taskExecutor) {
final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext();
final int workflowInstanceId = taskExecutionContext.getWorkflowInstanceId();
final IWorkflowExecutionRunnable workflowExecutionRunnable = IWorkflowRepository.get(workflowInstanceId);
if (workflowExecutionRunnable == null) {
throw new LogicTaskInitializeException(
"Cannot find the WorkflowExecuteRunnable by : " + workflowInstanceId);
}
return new SubWorkflowLogicTask(taskExecutionContext, workflowExecutionRunnable, applicationContext);
final IWorkflowExecutionRunnable workflowExecutionRunnable = workflowRepository.get(workflowInstanceId);
return new SubWorkflowLogicTask(
taskExecutionContext,
workflowExecutionRunnable,
taskExecutor,
applicationContext);
}
@Override

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskRuntimeContext.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow;
import lombok.AllArgsConstructor;
import lombok.Builder;

31
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowTracker.java

@ -15,56 +15,47 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction;
import java.time.Duration;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class SubWorkflowAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction {
private static final Duration SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL = Duration.ofSeconds(10);
public class SubWorkflowTracker {
private final WorkflowInstanceDao workflowInstanceDao;
private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext;
public SubWorkflowAsyncTaskExecuteFunction(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext,
public SubWorkflowTracker(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext,
final WorkflowInstanceDao workflowInstanceDao) {
this.subWorkflowLogicTaskRuntimeContext = subWorkflowLogicTaskRuntimeContext;
this.workflowInstanceDao = workflowInstanceDao;
}
@Override
public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
public @NonNull TaskExecutionStatus getSubWorkflowState() {
final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId();
final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId);
if (subWorkflowInstance == null) {
log.info("Cannot find the SubWorkflow instance: {}, maybe it has been deleted", subWorkflowInstanceId);
return AsyncTaskExecutionStatus.FAILED;
return TaskExecutionStatus.FAILURE;
}
switch (subWorkflowInstance.getState()) {
case PAUSE:
return AsyncTaskExecutionStatus.PAUSE;
return TaskExecutionStatus.PAUSE;
case STOP:
return AsyncTaskExecutionStatus.KILL;
return TaskExecutionStatus.KILL;
case SUCCESS:
return AsyncTaskExecutionStatus.SUCCESS;
return TaskExecutionStatus.SUCCESS;
case FAILURE:
return AsyncTaskExecutionStatus.FAILED;
return TaskExecutionStatus.FAILURE;
default:
return AsyncTaskExecutionStatus.RUNNING;
return TaskExecutionStatus.RUNNING_EXECUTION;
}
}
@Override
public @NonNull Duration getAsyncTaskStateCheckInterval() {
return SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL;
}
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/trigger/SubWorkflowManualTrigger.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.subworkflow.trigger;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow.trigger;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;

40
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTask.java

@ -15,20 +15,19 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.switchtask;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.switchtask;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext;
import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask;
import org.apache.dolphinscheduler.server.master.utils.SwitchTaskUtils;
import org.apache.commons.collections4.CollectionUtils;
@ -43,29 +42,25 @@ import lombok.extern.slf4j.Slf4j;
import com.fasterxml.jackson.core.type.TypeReference;
@Slf4j
public class SwitchLogicTask extends BaseSyncLogicTask<SwitchParameters> {
public static final String TASK_TYPE = "SWITCH";
public class SwitchLogicTask extends AbstractLogicTask<SwitchParameters> {
private final IWorkflowExecutionRunnable workflowExecutionRunnable;
private final TaskInstance taskInstance;
public SwitchLogicTask(IWorkflowExecutionRunnable workflowExecutionRunnable,
TaskExecutionContext taskExecutionContext) {
super(workflowExecutionRunnable,
taskExecutionContext,
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SwitchParameters>() {
}));
super(taskExecutionContext);
this.workflowExecutionRunnable = workflowExecutionRunnable;
this.taskInstance = workflowExecutionRunnable
.getWorkflowExecuteContext()
.getWorkflowExecutionGraph()
.getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId())
.getTaskInstance();
onTaskRunning();
}
@Override
public void handle() throws MasterTaskExecuteException {
public void start() {
if (CollectionUtils.isEmpty(taskParameters.getSwitchResult().getDependTaskList())) {
// If the branch is empty then will go into the default branch
// This case shouldn't happen, we can directly throw exception and forbid the user to set branch
@ -76,8 +71,9 @@ public class SwitchLogicTask extends BaseSyncLogicTask<SwitchParameters> {
}
checkIfBranchExist(taskParameters.getNextBranch());
taskInstance.setTaskParams(JSONUtils.toJsonString(taskParameters));
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
log.info("Switch task execute finished: {}", taskExecutionContext.getCurrentExecutionStatus().name());
onTaskSuccess();
log.info("Switch task execute finished");
}
private void moveToDefaultBranch() {
@ -144,4 +140,20 @@ public class SwitchLogicTask extends BaseSyncLogicTask<SwitchParameters> {
.orElse(null);
}
@Override
public void pause() {
log.info("The SwitchTask does not support pause operation");
}
@Override
public void kill() {
log.info("The SwitchTask does not support kill operation");
}
@Override
public ITaskParameterDeserializer<SwitchParameters> getTaskParameterDeserializer() {
return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference<SwitchParameters>() {
});
}
}

11
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTaskPluginFactory.java

@ -15,13 +15,15 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task.switchtask;
package org.apache.dolphinscheduler.server.master.engine.executor.plugin.switchtask;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
import lombok.extern.slf4j.Slf4j;
@ -36,7 +38,8 @@ public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory<Swi
private IWorkflowRepository IWorkflowRepository;
@Override
public SwitchLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
public SwitchLogicTask createLogicTask(final ITaskExecutor taskExecutor) throws LogicTaskInitializeException {
final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext();
final int workflowInstanceId = taskExecutionContext.getWorkflowInstanceId();
IWorkflowExecutionRunnable workflowExecutionRunnable =
IWorkflowRepository.get(workflowInstanceId);
@ -49,6 +52,6 @@ public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory<Swi
@Override
public String getTaskType() {
return SwitchLogicTask.TASK_TYPE;
return SwitchLogicTaskChannelFactory.NAME;
}
}

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

@ -22,9 +22,9 @@ import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.condition.ConditionLogicTask;
import org.apache.dolphinscheduler.server.master.engine.executor.plugin.switchtask.SwitchLogicTask;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.condition.ConditionLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.switchtask.SwitchLogicTask;
import org.apache.commons.collections4.CollectionUtils;

18
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/SystemEventBus.java

@ -23,8 +23,6 @@ import org.apache.dolphinscheduler.server.master.engine.system.event.AbstractSys
import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.AbstractWorkflowLifecycleLifecycleEvent;
import java.util.concurrent.LinkedBlockingQueue;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@ -37,24 +35,12 @@ import org.springframework.stereotype.Component;
@Component
public class SystemEventBus extends AbstractDelayEventBus<AbstractSystemEvent> {
private final LinkedBlockingQueue<AbstractSystemEvent> eventChannel = new LinkedBlockingQueue<>();
public void publish(final AbstractSystemEvent event) {
try {
eventChannel.put(event);
super.publish(event);
log.info("Published SystemEvent: {}", event);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new IllegalStateException("The thread has been interrupted", e);
}
}
public AbstractSystemEvent take() throws InterruptedException {
return eventChannel.take();
return delayEventQueue.take();
}
public boolean isEmpty() {
return eventChannel.isEmpty();
}
}

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/AbstractSystemEvent.java

@ -23,6 +23,10 @@ import java.util.Date;
public abstract class AbstractSystemEvent extends AbstractDelayEvent {
public AbstractSystemEvent() {
super();
}
public AbstractSystemEvent(long delayTime) {
super(delayTime);
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java

@ -29,7 +29,7 @@ public class GlobalMasterFailoverEvent extends AbstractSystemEvent {
private final Date eventTime;
public GlobalMasterFailoverEvent(Date eventTime) {
super(eventTime.getTime());
super();
this.eventTime = eventTime;
}

33
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java

@ -19,13 +19,31 @@ package org.apache.dolphinscheduler.server.master.engine.task.client;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskPauseException;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskReassignMasterHostException;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.task.executor.TaskEngine;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
/**
* The client of task executor, used to communicate with task executor.
* The client used to communicate with {@link TaskEngine}.
*/
public interface ITaskExecutorClient {
/**
* Dispatch the task to task executor.
*
* @throws TaskDispatchException If dispatch failed or error occurs.
*/
void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException;
/**
* Reassign the workflow instance host from task executor.
*
* @throws TaskReassignMasterHostException If an error occurs.
*/
boolean reassignWorkflowInstanceHost(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskReassignMasterHostException;
/**
* Pause task from task executor.
* <p> This method is not a sync method, it will return immediately after send a kill request to remote executor and receive a response.
@ -34,7 +52,7 @@ public interface ITaskExecutorClient {
*
* @throws TaskPauseException If an error occurs.
*/
void pause(ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException;
void pause(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException;
/**
* Kill task from task executor.
@ -44,5 +62,14 @@ public interface ITaskExecutorClient {
*
* @throws TaskKillException If an error occurs.
*/
void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException;
void kill(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException;
/**
* Send TaskExecutorLifecycleEventAck to TaskEngine.
* <p> This method will not throw exception, once send ack failed, the executor engine will retry.
*/
void ackTaskExecutorLifecycleEvent(
final ITaskExecutionRunnable taskExecutionRunnable,
final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck);
}

27
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java

@ -18,6 +18,8 @@
package org.apache.dolphinscheduler.server.master.engine.task.client;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
/**
* The interface of task executor client delegator. It is used to send operation to task executor server.
@ -27,9 +29,32 @@ import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecu
*/
public interface ITaskExecutorClientDelegator {
void dispatch(final ITaskExecutionRunnable taskExecutionRunnable);
/**
* Dispatch the task to task executor.
*
* @throws TaskDispatchException If dispatch failed
*/
void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException;
/**
* Take over the task from task executor.
*/
boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable);
/**
* Pause the task, this method doesn't guarantee the task is paused success.
*/
void pause(final ITaskExecutionRunnable taskExecutionRunnable);
/**
* Kill the task, this method doesn't guarantee the task is killed success.
*/
void kill(final ITaskExecutionRunnable taskExecutionRunnable);
/**
* Ack the task executor lifecycle event.
*/
void ackTaskExecutorLifecycleEvent(
final ITaskExecutionRunnable taskExecutionRunnable,
final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck);
}

31
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskOperator.java

@ -1,31 +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.engine.task.client;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
public interface ITaskOperator {
void dispatch(ITaskExecutionRunnable taskExecutionRunnable);
void pause(ITaskExecutionRunnable taskExecutionRunnable);
void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException;
}

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

@ -21,27 +21,57 @@ import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseRequest;
import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseResponse;
import org.apache.dolphinscheduler.extract.master.ILogicTaskExecutorOperator;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse;
import org.apache.commons.lang3.StringUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDelegator {
@Autowired
private MasterConfig masterConfig;
@Override
public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) {
public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException {
final String logicTaskExecutorAddress = masterConfig.getMasterAddress();
final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext();
taskExecutionContext.setHost(logicTaskExecutorAddress);
taskExecutionRunnable.getTaskInstance().setHost(logicTaskExecutorAddress);
final TaskExecutorDispatchResponse logicTaskDispatchResponse = Clients
.withService(ILogicTaskExecutorOperator.class)
.withHost(logicTaskExecutorAddress)
.dispatchTask(TaskExecutorDispatchRequest.of(taskExecutionContext));
if (!logicTaskDispatchResponse.isDispatchSuccess()) {
throw new TaskDispatchException(
String.format("Dispatch LogicTask to %s failed, response is: %s",
taskExecutionContext.getHost(), logicTaskDispatchResponse));
}
}
@Override
public boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable) {
// The Logic Task doesn't support take-over, since the logic task is not executed on the worker.
return false;
}
@Override
@ -51,10 +81,10 @@ public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDele
final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final LogicTaskPauseResponse pauseResponse = Clients
.withService(ILogicTaskInstanceOperator.class)
final TaskExecutorPauseResponse pauseResponse = Clients
.withService(ILogicTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.pauseLogicTask(new LogicTaskPauseRequest(taskInstance.getId()));
.pauseTask(TaskExecutorPauseRequest.of(taskInstance.getId()));
if (pauseResponse.isSuccess()) {
log.info("Pause task {} on executor {} successfully", taskName, executorHost);
} else {
@ -69,14 +99,29 @@ public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDele
final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final LogicTaskKillResponse killResponse = Clients
.withService(ILogicTaskInstanceOperator.class)
final TaskExecutorKillResponse killResponse = Clients
.withService(ILogicTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.killLogicTask(new LogicTaskKillRequest(taskInstance.getId()));
.killTask(TaskExecutorKillRequest.of(taskInstance.getId()));
if (killResponse.isSuccess()) {
log.info("Kill task {} on executor {} successfully", taskName, executorHost);
} else {
log.warn("Kill task {} on executor {} failed with response {}", taskName, executorHost, killResponse);
}
}
@Override
public void ackTaskExecutorLifecycleEvent(
final ITaskExecutionRunnable taskExecutionRunnable,
final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance();
final String executorHost = taskInstance.getHost();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
Clients
.withService(ILogicTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.ackTaskExecutorLifecycleEvent(taskExecutorLifecycleEventAck);
}
}

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

@ -21,26 +21,108 @@ import static com.google.common.base.Preconditions.checkArgument;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseResponse;
import org.apache.dolphinscheduler.extract.base.utils.Host;
import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterResponse;
import org.apache.commons.lang3.StringUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientDelegator {
@Autowired
private MasterConfig masterConfig;
@Autowired
private IWorkerLoadBalancer workerLoadBalancer;
@Override
public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException {
final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext();
final String taskName = taskExecutionContext.getTaskName();
final String physicalTaskExecutorAddress = workerLoadBalancer
.select(taskExecutionContext.getWorkerGroup())
.map(Host::of)
.map(Host::getAddress)
.orElseThrow(() -> new TaskDispatchException(
String.format("Cannot find the host to dispatch Task[id=%s, name=%s]",
taskExecutionContext.getTaskInstanceId(), taskName)));
taskExecutionContext.setHost(physicalTaskExecutorAddress);
taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress);
try {
final TaskExecutorDispatchResponse taskExecutorDispatchResponse = Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(physicalTaskExecutorAddress)
.dispatchTask(TaskExecutorDispatchRequest.of(taskExecutionRunnable.getTaskExecutionContext()));
if (!taskExecutorDispatchResponse.isDispatchSuccess()) {
throw new TaskDispatchException(
"Dispatch task: " + taskName + " to " + physicalTaskExecutorAddress + " failed: "
+ taskExecutorDispatchResponse);
}
} catch (TaskDispatchException e) {
throw e;
} catch (Exception e) {
throw new TaskDispatchException(
"Dispatch task: " + taskName + " to " + physicalTaskExecutorAddress + " failed", e);
}
}
@Override
public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) {
public boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable) {
final String taskName = taskExecutionRunnable.getName();
checkArgument(taskExecutionRunnable.isTaskInstanceInitialized(),
"Task " + taskName + "is not initialized cannot take-over");
final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance();
final String taskExecutorHost = taskInstance.getHost();
if (StringUtils.isEmpty(taskExecutorHost)) {
log.debug(
"The task executor: {} host is empty, cannot take-over, this might caused by the task hasn't dispatched",
taskName);
return false;
}
final TaskExecutorReassignMasterRequest taskExecutorReassignMasterRequest =
TaskExecutorReassignMasterRequest.builder()
.taskInstanceId(taskInstance.getId())
.workflowHost(masterConfig.getMasterAddress())
.build();
final TaskExecutorReassignMasterResponse taskExecutorReassignMasterResponse =
Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.reassignWorkflowInstanceHost(taskExecutorReassignMasterRequest);
boolean success = taskExecutorReassignMasterResponse.isSuccess();
if (success) {
log.info("Reassign master host {} to {} successfully", taskExecutorHost, taskName);
} else {
log.info("Reassign master host {} on {} failed with response {}",
taskExecutorHost,
taskName,
taskExecutorReassignMasterResponse);
}
return success;
}
@Override
@ -50,10 +132,10 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD
final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final TaskInstancePauseResponse pauseResponse = Clients
.withService(ITaskInstanceOperator.class)
final TaskExecutorPauseResponse pauseResponse = Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(taskInstance.getHost())
.pauseTask(new TaskInstancePauseRequest(taskInstance.getId()));
.pauseTask(TaskExecutorPauseRequest.of(taskInstance.getId()));
if (pauseResponse.isSuccess()) {
log.info("Pause task {} on executor {} successfully", taskName, executorHost);
} else {
@ -68,14 +150,28 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD
final String taskName = taskInstance.getName();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
final TaskInstanceKillResponse killResponse = Clients
.withService(ITaskInstanceOperator.class)
final TaskExecutorKillResponse killResponse = Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(executorHost)
.killTask(new TaskInstanceKillRequest(taskInstance.getId()));
.killTask(TaskExecutorKillRequest.of(taskInstance.getId()));
if (killResponse.isSuccess()) {
log.info("Kill task {} on executor {} successfully", taskName, executorHost);
} else {
log.warn("Kill task {} on executor {} failed with response {}", taskName, executorHost, killResponse);
}
}
@Override
public void ackTaskExecutorLifecycleEvent(final ITaskExecutionRunnable taskExecutionRunnable,
final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance();
final String executorHost = taskInstance.getHost();
checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty");
Clients
.withService(IPhysicalTaskExecutorOperator.class)
.withHost(executorHost)
.ackPhysicalTaskExecutorLifecycleEvent(taskExecutorLifecycleEventAck);
}
}

53
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java

@ -23,7 +23,14 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskPauseException;
import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskReassignMasterHostException;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.commons.lang3.StringUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@ -31,6 +38,7 @@ import org.springframework.stereotype.Component;
/**
* The client of task executor, used to communicate with task executor.
*/
@Slf4j
@Component
public class TaskExecutorClient implements ITaskExecutorClient {
@ -41,7 +49,31 @@ public class TaskExecutorClient implements ITaskExecutorClient {
private PhysicalTaskExecutorClientDelegator physicalTaskExecutorClientDelegator;
@Override
public void pause(ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException {
public void dispatch(ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException {
try {
getTaskExecutorClientDelegator(taskExecutionRunnable).dispatch(taskExecutionRunnable);
} catch (TaskDispatchException taskDispatchException) {
throw taskDispatchException;
} catch (Exception ex) {
throw new TaskDispatchException("Dispatch task: " + taskExecutionRunnable.getName() + " to executor failed",
ex);
}
}
@Override
public boolean reassignWorkflowInstanceHost(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskReassignMasterHostException {
try {
return getTaskExecutorClientDelegator(taskExecutionRunnable)
.reassignMasterHost(taskExecutionRunnable);
} catch (Exception ex) {
throw new TaskReassignMasterHostException(
"Take over task: " + taskExecutionRunnable.getName() + " from executor failed",
ex);
}
}
@Override
public void pause(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException {
try {
getTaskExecutorClientDelegator(taskExecutionRunnable).pause(taskExecutionRunnable);
} catch (Exception ex) {
@ -51,7 +83,7 @@ public class TaskExecutorClient implements ITaskExecutorClient {
}
@Override
public void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException {
public void kill(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException {
try {
getTaskExecutorClientDelegator(taskExecutionRunnable).kill(taskExecutionRunnable);
} catch (Exception ex) {
@ -59,6 +91,23 @@ public class TaskExecutorClient implements ITaskExecutorClient {
}
}
@Override
public void ackTaskExecutorLifecycleEvent(
final ITaskExecutionRunnable taskExecutionRunnable,
final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
try {
if (StringUtils.isEmpty(taskExecutionRunnable.getTaskInstance().getHost())) {
log.info("The task: {} is didn't dispatched to executor, skip ack taskExecutorLifecycleEventAck: {}",
taskExecutionRunnable.getName(), taskExecutorLifecycleEventAck);
return;
}
getTaskExecutorClientDelegator(taskExecutionRunnable)
.ackTaskExecutorLifecycleEvent(taskExecutionRunnable, taskExecutorLifecycleEventAck);
} catch (Exception ex) {
log.error("Send taskExecutorLifecycleEventAck: {} failed", taskExecutorLifecycleEventAck, ex);
}
}
private ITaskExecutorClientDelegator getTaskExecutorClientDelegator(final ITaskExecutionRunnable taskExecutionRunnable) {
final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance();
checkArgument(taskInstance != null, "taskType cannot be empty");

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java

@ -38,6 +38,10 @@ public enum TaskLifecycleEventType implements ILifecycleEventType {
* The task instance is running at the target executor server.
*/
RUNNING,
/**
* The task instance's runtime context changed.
*/
RUNTIME_CONTEXT_CHANGED,
/**
* Do Timeout strategy of the task instance.
*/

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java

@ -37,8 +37,6 @@ public class TaskRunningLifecycleEvent extends AbstractTaskLifecycleEvent {
private final String logPath;
private final String runtimeContext;
private final Date startTime;
@Override

50
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRuntimeContextChangedEvent.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.server.master.engine.task.lifecycle.event;
import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType;
import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Getter;
@Getter
@Builder
@AllArgsConstructor
public class TaskRuntimeContextChangedEvent extends AbstractTaskLifecycleEvent {
private final ITaskExecutionRunnable taskExecutionRunnable;
private final String runtimeContext;
@Override
public ILifecycleEventType getEventType() {
return TaskLifecycleEventType.RUNTIME_CONTEXT_CHANGED;
}
@Override
public String toString() {
return "TaskRunningLifecycleEvent{" +
"task=" + taskExecutionRunnable.getName() +
", runtimeContext=" + runtimeContext +
'}';
}
}

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

Loading…
Cancel
Save