diff --git a/docs/docs/en/architecture/configuration.md b/docs/docs/en/architecture/configuration.md
index fb8613d388..5794152966 100644
--- a/docs/docs/en/architecture/configuration.md
+++ b/docs/docs/en/architecture/configuration.md
@@ -276,29 +276,21 @@ 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.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 |
-| master.command-fetch-strategy.config.fetch-size | 10 | The number of commands fetched by master |
+| Parameters | Default value | Description |
+|-----------------------------------------------------------------------------|------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------|
+| master.listen-port | 5678 | master listen port |
+| 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.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.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 |
+| master.command-fetch-strategy.config.fetch-size | 10 | The number of commands fetched by master |
### Worker Server related configuration
@@ -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`. |
diff --git a/docs/docs/en/guide/upgrade/incompatible.md b/docs/docs/en/guide/upgrade/incompatible.md
index 91abd04811..12dd7fa356 100644
--- a/docs/docs/en/guide/upgrade/incompatible.md
+++ b/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)
diff --git a/docs/docs/zh/architecture/configuration.md b/docs/docs/zh/architecture/configuration.md
index c76e869f72..d167bba45e 100644
--- a/docs/docs/zh/architecture/configuration.md
+++ b/docs/docs/zh/architecture/configuration.md
@@ -303,24 +303,22 @@ 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后自动创建。 |
-| worker.server-load-protection.enabled | true | 是否开启系统保护策略 |
-| worker.server-load-protection.max-system-cpu-usage-percentage-thresholds | 0.7 | worker最大系统cpu使用值,只有当前系统cpu使用值低于最大系统cpu使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的操作系统CPU |
-| worker.server-load-protection.max-jvm-cpu-usage-percentage-thresholds | 0.7 | worker最大JVM cpu使用值,只有当前JVM cpu使用值低于最大JVM cpu使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的JVM CPU |
-| worker.server-load-protection.max-system-memory-usage-percentage-thresholds | 0.7 | worker最大系统 内存使用值,只有当前系统内存使用值低于最大系统内存使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的操作系统内存 |
-| 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.tenant-config.auto-create-tenant-enabled | true | 租户对应于系统的用户,由worker提交作业.如果系统没有该用户,则在参数worker.tenant.auto.create为true后自动创建。 |
-| worker.tenant-config.default-tenant-enabled | false | 如果设置为true, 将会使用worker服务启动用户作为 `default` 租户。 |
+| 参数 | 默认值 | 描述 |
+|-----------------------------------------------------------------------------|-----------|-----------------------------------------------------------------------------------------|
+| worker.listen-port | 1234 | worker监听端口 |
+| worker.max-heartbeat-interval | 10s | worker最大心跳间隔 |
+| worker.host-weight | 100 | 派发任务时,worker主机的权重 |
+| worker.tenant-auto-create | true | 租户对应于系统的用户,由worker提交作业.如果系统没有该用户,则在参数worker.tenant.auto.create为true后自动创建。 |
+| worker.server-load-protection.enabled | true | 是否开启系统保护策略 |
+| worker.server-load-protection.max-system-cpu-usage-percentage-thresholds | 0.7 | worker最大系统cpu使用值,只有当前系统cpu使用值低于最大系统cpu使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的操作系统CPU |
+| worker.server-load-protection.max-jvm-cpu-usage-percentage-thresholds | 0.7 | worker最大JVM cpu使用值,只有当前JVM cpu使用值低于最大JVM cpu使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的JVM CPU |
+| worker.server-load-protection.max-system-memory-usage-percentage-thresholds | 0.7 | worker最大系统 内存使用值,只有当前系统内存使用值低于最大系统内存使用值,worker服务才能接收任务. 默认值为0.7: 会使用70%的操作系统内存 |
+| 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.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` 租户。 |
## Alert Server相关配置
diff --git a/docs/docs/zh/guide/upgrade/incompatible.md b/docs/docs/zh/guide/upgrade/incompatible.md
index 4ba7d4bf95..412a66e4b3 100644
--- a/docs/docs/zh/guide/upgrade/incompatible.md
+++ b/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)
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
index 957aca912f..372ee35b6d 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
+++ b/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;
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
index c6ced818c6..306cf91de2 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
+++ b/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) {
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
index 932da89d6b..1c3cfc86b7 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
+++ b/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));
}
@@ -43,9 +49,10 @@ public class ThreadUtils {
* Create a daemon scheduler 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
+ * @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));
}
diff --git a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEvent.java b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEvent.java
index bc57aef9e9..c4c61b9055 100644
--- a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEvent.java
+++ b/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.
*
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(final long delayTime) {
+ this(delayTime, System.nanoTime());
}
- public AbstractDelayEvent(long delayTime) {
- if (delayTime == 0) {
- this.triggerTimeInMillis = System.currentTimeMillis();
- } else {
- this.triggerTimeInMillis = System.currentTimeMillis() + delayTime;
- }
+ 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);
}
}
diff --git a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java
index 4d3c259e45..81809c546c 100644
--- a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java
+++ b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/AbstractDelayEventBus.java
@@ -37,6 +37,16 @@ public abstract class AbstractDelayEventBus implem
return Optional.ofNullable(delayEventQueue.poll());
}
+ @Override
+ public Optional peek() {
+ return Optional.ofNullable(delayEventQueue.peek());
+ }
+
+ @Override
+ public Optional remove() {
+ return Optional.ofNullable(delayEventQueue.remove());
+ }
+
@Override
public boolean isEmpty() {
return delayEventQueue.isEmpty();
diff --git a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java
index 8a8c458e6a..b2f6577a41 100644
--- a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java
+++ b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEventBus.java
@@ -54,6 +54,16 @@ public interface IEventBus {
*/
Optional 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 peek();
+
+ /**
+ * Remove the head event from the bus. This method will not block if the event bus is empty will return empty optional.
+ */
+ Optional remove();
+
/**
* Whether the bus is empty.
*/
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml b/dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml
index 0554ea3c3c..8a03faf409 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-base/pom.xml
@@ -45,27 +45,28 @@
org.apache.dolphinscheduler
dolphinscheduler-common
- ${project.version}
org.apache.dolphinscheduler
dolphinscheduler-meter
- ${project.version}
io.netty
netty-all
+
org.slf4j
slf4j-api
+
com.google.guava
guava
+
org.springframework
spring-context
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/serialize/JsonSerializer.java b/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/serialize/JsonSerializer.java
index 56e19fe8b2..3450192f62 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/serialize/JsonSerializer.java
+++ b/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 object type
- * @return byte array
- */
public static byte[] serialize(T obj) {
if (obj == null) {
return null;
@@ -79,44 +69,14 @@ public class JsonSerializer {
}
}
- /**
- * serialize to string
- *
- * @param obj object
- * @param object type
- * @return string
- */
- public static 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 deserialize type
- * @return deserialize type
- */
+ @SneakyThrows
public static T deserialize(byte[] src, Class 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;
- }
+ return objectMapper.readValue(json, clazz);
}
}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/NettyRemotingServer.java b/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/NettyRemotingServer.java
index 9ebf802b1e..246e11735c 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/NettyRemotingServer.java
+++ b/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);
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml b/dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml
index d7d339ff9f..f763ec22c0 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-common/pom.xml
@@ -27,14 +27,12 @@
dev-SNAPSHOT
- org.apache.dolphinscheduler
dolphinscheduler-extract-common
org.apache.dolphinscheduler
dolphinscheduler-extract-base
- ${project.version}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-common/src/main/java/org/apache/dolphinscheduler/extract/common/ILogService.java b/dolphinscheduler-extract/dolphinscheduler-extract-common/src/main/java/org/apache/dolphinscheduler/extract/common/ILogService.java
index c602f15bb2..4484e51942 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-common/src/main/java/org/apache/dolphinscheduler/extract/common/ILogService.java
+++ b/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);
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml b/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
index 955bcd0a12..e210fd9056 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
@@ -30,21 +30,27 @@
dolphinscheduler-extract-master
+
org.apache.dolphinscheduler
dolphinscheduler-extract-common
- ${project.version}
+
org.apache.dolphinscheduler
dolphinscheduler-extract-base
- ${project.version}
+
org.apache.dolphinscheduler
dolphinscheduler-task-api
- ${project.version}
+
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+
+
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskExecutorOperator.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskExecutorOperator.java
new file mode 100644
index 0000000000..1344a29ce1
--- /dev/null
+++ b/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);
+
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskInstanceOperator.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskInstanceOperator.java
deleted file mode 100644
index f85200b7a2..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ILogicTaskInstanceOperator.java
+++ /dev/null
@@ -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);
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutionEventListener.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutionEventListener.java
deleted file mode 100644
index 7ebd1310bd..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutionEventListener.java
+++ /dev/null
@@ -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);
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutorEventListener.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskExecutorEventListener.java
new file mode 100644
index 0000000000..86b8f66568
--- /dev/null
+++ b/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);
+
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/TaskExecutorEventRemoteReporterClient.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/TaskExecutorEventRemoteReporterClient.java
new file mode 100644
index 0000000000..955e44c662
--- /dev/null
+++ b/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);
+ }
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/ITaskExecutionEvent.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/ITaskExecutionEvent.java
index a1df0829a5..8d9f4f7a90 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/ITaskExecutionEvent.java
+++ b/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,
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerRequest.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerRequest.java
deleted file mode 100644
index 7f2b16a530..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerRequest.java
+++ /dev/null
@@ -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 startParams;
-
- private String tenantCode;
-
- private int dryRun;
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionSuccessEvent.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionSuccessEvent.java
deleted file mode 100644
index a7dd1c8909..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionSuccessEvent.java
+++ /dev/null
@@ -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;
- }
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml b/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
index 0a42b73a1d..57d3857292 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
@@ -30,11 +30,12 @@
dolphinscheduler-extract-worker
+
org.apache.dolphinscheduler
dolphinscheduler-extract-common
- ${project.version}
+
org.apache.dolphinscheduler
dolphinscheduler-extract-base
@@ -45,6 +46,11 @@
dolphinscheduler-task-api
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+
+
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/IPhysicalTaskExecutorOperator.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/IPhysicalTaskExecutorOperator.java
new file mode 100644
index 0000000000..b0cf4538ea
--- /dev/null
+++ b/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);
+
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceExecutionEventAckListener.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceExecutionEventAckListener.java
deleted file mode 100644
index fda644110f..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceExecutionEventAckListener.java
+++ /dev/null
@@ -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);
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceOperator.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceOperator.java
deleted file mode 100644
index 43d6a39b01..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/ITaskInstanceOperator.java
+++ /dev/null
@@ -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);
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionFailedEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionFailedEventAck.java
deleted file mode 100644
index 3c02bbe461..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionFailedEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionKilledEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionKilledEventAck.java
deleted file mode 100644
index 38b6441c1f..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionKilledEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionPausedEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionPausedEventAck.java
deleted file mode 100644
index fee22e9431..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionPausedEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionSuccessEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionSuccessEventAck.java
deleted file mode 100644
index bcaba1d4fd..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskExecutionSuccessEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchResponse.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchResponse.java
deleted file mode 100644
index 8845ed9081..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchResponse.java
+++ /dev/null
@@ -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);
- }
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionDispatchedEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionDispatchedEventAck.java
deleted file mode 100644
index 088058284a..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionDispatchedEventAck.java
+++ /dev/null
@@ -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);
- }
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionFinishEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionFinishEventAck.java
deleted file mode 100644
index 90ae49ab0e..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionFinishEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionRunningEventAck.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionRunningEventAck.java
deleted file mode 100644
index aaeb4eb690..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceExecutionRunningEventAck.java
+++ /dev/null
@@ -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);
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillResponse.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillResponse.java
deleted file mode 100644
index 2999a299bd..0000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillResponse.java
+++ /dev/null
@@ -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 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;
- }
-
-}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointRequest.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointRequest.java
index 3d649fa8d8..c86e70268f 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointRequest.java
+++ b/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
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointResponse.java b/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointResponse.java
index 82093e1cf9..9d70b002f0 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceTriggerSavepointResponse.java
+++ b/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
diff --git a/dolphinscheduler-master/pom.xml b/dolphinscheduler-master/pom.xml
index 81ca7171bc..8283e9e539 100644
--- a/dolphinscheduler-master/pom.xml
+++ b/dolphinscheduler-master/pom.xml
@@ -107,7 +107,11 @@
org.apache.dolphinscheduler
dolphinscheduler-eventbus
- ${project.version}
+
+
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
index 4cffd3e372..7695f7adc3 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
+++ b/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;
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java
similarity index 79%
rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java
index a1702b588e..46c5405ee3 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java
+++ b/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;
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
index bc1cfef2ec..76be92e36c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
+++ b/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 +
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/IWorkflowRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/IWorkflowRepository.java
index 026eda7e45..37084d3098 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/IWorkflowRepository.java
+++ b/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();
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java
index 905a5d2fa8..19a20dff9a 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/TaskGroupCoordinator.java
+++ b/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());
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowCacheRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowCacheRepository.java
index 85d01fb51e..6f069f8eee 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowCacheRepository.java
+++ b/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();
- }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java
index 76ca89dd57..75bc6b4527 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java
+++ b/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
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorker.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorker.java
index 5191d32a51..905f4b7842 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorker.java
+++ b/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 getWaitingFireWorkflowExecutionRunnables() {
if (MapUtils.isEmpty(registeredWorkflowExecuteRunnableMap)) {
return Collections.emptyList();
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorkers.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorkers.java
index 5542e7ae4d..fdc038eb32 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEventBusFireWorkers.java
+++ b/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();
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/IdSlotBasedCommandFetcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/IdSlotBasedCommandFetcher.java
index cfd13b4e45..f76a0dfa47 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/command/IdSlotBasedCommandFetcher.java
+++ b/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;
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskReassignMasterHostException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskReassignMasterHostException.java
new file mode 100644
index 0000000000..0e44e7af64
--- /dev/null
+++ b/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);
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineDelegator.java
new file mode 100644
index 0000000000..da76deec16
--- /dev/null
+++ b/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");
+ }
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java
new file mode 100644
index 0000000000..d098946bbb
--- /dev/null
+++ b/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);
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutor.java
new file mode 100644
index 0000000000..4dafabfe4d
--- /dev/null
+++ b/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() +
+ '}';
+ }
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
similarity index 60%
rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
index c1c460acd6..4d245a164d 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java
+++ b/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;
- }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerProvider.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerProvider.java
new file mode 100644
index 0000000000..709f9d61df
--- /dev/null
+++ b/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;
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
similarity index 53%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
index 51bda7edbe..d696633cba 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java
+++ b/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 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);
}
-
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventRemoteReporterClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventRemoteReporterClient.java
new file mode 100644
index 0000000000..4cb7551ba9
--- /dev/null
+++ b/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 {
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorFactory.java
new file mode 100644
index 0000000000..355fdf76bc
--- /dev/null
+++ b/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));
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java
new file mode 100644
index 0000000000..e9efc60a2d
--- /dev/null
+++ b/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);
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventReporter.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventReporter.java
new file mode 100644
index 0000000000..41a80ba716
--- /dev/null
+++ b/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);
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorRepository.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorRepository.java
new file mode 100644
index 0000000000..510ad88f19
--- /dev/null
+++ b/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 {
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/AbstractLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/AbstractLogicTask.java
new file mode 100644
index 0000000000..98f78aa452
--- /dev/null
+++ b/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 implements ILogicTask {
+
+ 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;
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTask.java
similarity index 74%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTask.java
index 806792f45a..5052b0b73c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java
+++ b/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 {
+
+ void start() throws MasterTaskExecuteException;
void pause() throws MasterTaskExecuteException;
void kill() throws MasterTaskExecuteException;
- AbstractParameters getTaskParameters();
+ TaskExecutionStatus getTaskExecutionState();
- TaskExecutionContext getTaskExecutionContext();
+ ITaskParameterDeserializer getTaskParameterDeserializer();
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTaskPluginFactory.java
similarity index 69%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ILogicTaskPluginFactory.java
index 08b2285184..7beacc204f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java
+++ b/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 {
+public interface ILogicTaskPluginFactory> {
- T createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException;
+ T createLogicTask(final ITaskExecutor taskExecutor) throws LogicTaskInitializeException;
String getTaskType();
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java
similarity index 83%
rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java
index e8865c985c..31d4aa5bf0 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java
+++ b/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 {
+public interface ITaskParameterDeserializer {
- Y operate(X x);
+ T deserialize(String taskParamsJson);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/LogicTaskPluginFactoryBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/LogicTaskPluginFactoryBuilder.java
similarity index 68%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/LogicTaskPluginFactoryBuilder.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/LogicTaskPluginFactoryBuilder.java
index 7d6cc7260c..5642e407aa 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/LogicTaskPluginFactoryBuilder.java
+++ b/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 logicTaskPluginFactoryMap = new ConcurrentHashMap<>();
+ private final Map>> logicTaskPluginFactoryMap =
+ new ConcurrentHashMap<>();
- public LogicTaskPluginFactoryBuilder(List logicTaskPluginFactories) {
+ public LogicTaskPluginFactoryBuilder(List>> 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);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTask.java
similarity index 76%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTask.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTask.java
index 6088ff53ea..dcfca074e4 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTask.java
+++ b/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 {
-
- public static final String TASK_TYPE = "CONDITIONS";
+public class ConditionLogicTask extends AbstractLogicTask {
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() {
- }));
+ 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
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 getTaskParameterDeserializer() {
+ return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference() {
+ });
+ }
+
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTaskPluginFactory.java
similarity index 80%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/condition/ConditionLogicTaskPluginFactory.java
index dc797c6a68..7c4ee34895 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.java
+++ b/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);
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTask.java
similarity index 60%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTask.java
index 70ac2ba10b..3e4d7314f0 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java
+++ b/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 {
+public class DependentLogicTask extends AbstractLogicTask {
- 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
TaskInstanceDao taskInstanceDao,
WorkflowInstanceDao workflowInstanceDao,
IWorkflowExecutionRunnable workflowExecutionRunnable) {
- super(taskExecutionContext,
- JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() {
- }));
- 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 getTaskParameterDeserializer() {
+ return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference() {
+ });
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTaskPluginFactory.java
similarity index 82%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentLogicTaskPluginFactory.java
index 31c4f67bf6..6c4f35b1fb 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java
+++ b/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;
}
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentAsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentTaskTracker.java
similarity index 89%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentAsyncTaskExecuteFunction.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dependent/DependentTaskTracker.java
index bf8752a98d..f2cae5828e 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentAsyncTaskExecuteFunction.java
+++ b/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,13 +70,13 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
private final Map dependResultMap;
private final Map dependVarPoolPropertyMap;
- public DependentAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext,
- DependentParameters dependentParameters,
- ProjectDao projectDao,
- WorkflowDefinitionDao workflowDefinitionDao,
- TaskDefinitionDao taskDefinitionDao,
- TaskInstanceDao taskInstanceDao,
- WorkflowInstanceDao workflowInstanceDao) {
+ public DependentTaskTracker(TaskExecutionContext taskExecutionContext,
+ DependentParameters dependentParameters,
+ ProjectDao projectDao,
+ WorkflowDefinitionDao workflowDefinitionDao,
+ TaskDefinitionDao taskDefinitionDao,
+ TaskInstanceDao taskInstanceDao,
+ WorkflowInstanceDao workflowInstanceDao) {
this.taskExecutionContext = taskExecutionContext;
this.dependentParameters = dependentParameters;
this.projectDao = projectDao;
@@ -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());
- }
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicAsyncTaskExecuteFunction.java
similarity index 90%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunction.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicAsyncTaskExecuteFunction.java
index 9f2b483579..b8ba05fe7a 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunction.java
+++ b/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 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 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;
- }
-
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicCommandUtils.java
similarity index 97%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtils.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicCommandUtils.java
index 567cd7ff88..2dc76c9e72 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtils.java
+++ b/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;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTask.java
similarity index 86%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTask.java
index 6942260d8b..40fd7ab69b 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
+++ b/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 {
+public class DynamicLogicTask extends AbstractLogicTask {
public static final String TASK_TYPE = "DYNAMIC";
private final WorkflowInstanceDao workflowInstanceDao;
@@ -78,6 +79,8 @@ public class DynamicLogicTask extends BaseAsyncLogicTask {
private TaskInstance taskInstance;
+ private final TaskExecutionContext taskExecutionContext;
+
private boolean haveBeenCanceled = false;
public DynamicLogicTask(TaskExecutionContext taskExecutionContext,
@@ -87,9 +90,8 @@ public class DynamicLogicTask extends BaseAsyncLogicTask {
ProcessService processService,
WorkflowDefinitionMapper workflowDefinitionMapper,
CommandMapper commandMapper) {
- super(taskExecutionContext,
- JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() {
- }));
+ super(taskExecutionContext);
+ this.taskExecutionContext = taskExecutionContext;
this.workflowInstanceDao = workflowInstanceDao;
this.subWorkflowService = subWorkflowService;
this.processService = processService;
@@ -100,28 +102,27 @@ public class DynamicLogicTask extends BaseAsyncLogicTask {
this.taskInstance = taskInstanceDao.queryById(taskExecutionContext.getTaskInstanceId());
}
- @Override
- public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException {
- List
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+ ${project.version}
+
+
org.apache.dolphinscheduler
dolphinscheduler-yarn-aop
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
index 4618c5ad59..26c847ca21 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
@@ -27,21 +27,12 @@ import org.apache.dolphinscheduler.meter.metrics.MetricsProvider;
import org.apache.dolphinscheduler.meter.metrics.SystemMetrics;
import org.apache.dolphinscheduler.plugin.datasource.api.plugin.DataSourceProcessorProvider;
import org.apache.dolphinscheduler.plugin.storage.api.StorageConfiguration;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils;
import org.apache.dolphinscheduler.registry.api.RegistryConfiguration;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskEngineDelegator;
import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcServer;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
-
-import org.apache.commons.collections4.CollectionUtils;
-
-import java.util.Collection;
import javax.annotation.PostConstruct;
@@ -66,10 +57,10 @@ public class WorkerServer implements IStoppable {
private WorkerRpcServer workerRpcServer;
@Autowired
- private MessageRetryRunner messageRetryRunner;
+ private MetricsProvider metricsProvider;
@Autowired
- private MetricsProvider metricsProvider;
+ private PhysicalTaskEngineDelegator physicalTaskEngineDelegator;
/**
* worker server startup, not use web service
@@ -86,14 +77,17 @@ public class WorkerServer implements IStoppable {
@PostConstruct
public void run() {
ServerLifeCycleManager.toRunning();
+
this.workerRpcServer.start();
+
TaskPluginManager.loadTaskPlugin();
+
DataSourceProcessorProvider.initialize();
this.workerRegistryClient.setRegistryStoppable(this);
this.workerRegistryClient.start();
- this.messageRetryRunner.start();
+ this.physicalTaskEngineDelegator.start();
WorkerServerMetrics.registerWorkerCpuUsageGauge(() -> {
SystemMetrics systemMetrics = metricsProvider.getSystemMetrics();
@@ -126,17 +120,10 @@ public class WorkerServer implements IStoppable {
ThreadUtils.sleep(Constants.SERVER_CLOSE_WAIT_TIME.toMillis());
try (
- WorkerRpcServer closedWorkerRpcServer = workerRpcServer;
- WorkerRegistryClient closedRegistryClient = workerRegistryClient) {
+ final PhysicalTaskEngineDelegator ignore1 = physicalTaskEngineDelegator;
+ final WorkerRpcServer ignore2 = workerRpcServer;
+ final WorkerRegistryClient ignore3 = workerRegistryClient) {
log.info("Worker server is stopping, current cause : {}", cause);
- // todo: we need to remove this method
- // since for some task, we need to take-over the remote task after the worker restart
- // and if the worker crash, the `killAllRunningTasks` will not be execute, this will cause there exist two
- // kind of situation:
- // 1. If the worker is stop by kill, the tasks will be kill.
- // 2. If the worker is stop by kill -9, the tasks will not be kill.
- // So we don't need to kill the tasks.
- this.killAllRunningTasks();
} catch (Exception e) {
log.error("Worker server stop failed, current cause: {}", cause, e);
return;
@@ -153,26 +140,4 @@ public class WorkerServer implements IStoppable {
System.exit(1);
}
- public void killAllRunningTasks() {
- Collection workerTaskExecutors = WorkerTaskExecutorHolder.getAllTaskExecutor();
- if (CollectionUtils.isEmpty(workerTaskExecutors)) {
- return;
- }
- log.info("Worker begin to kill all cache task, task size: {}", workerTaskExecutors.size());
- int killNumber = 0;
- for (WorkerTaskExecutor workerTaskExecutor : workerTaskExecutors) {
- // kill task when it's not finished yet
- try {
- TaskExecutionContext taskExecutionContext = workerTaskExecutor.getTaskExecutionContext();
- LogUtils.setTaskInstanceIdMDC(taskExecutionContext.getTaskInstanceId());
- if (ProcessUtils.kill(taskExecutionContext)) {
- killNumber++;
- }
- } finally {
- LogUtils.removeWorkflowAndTaskInstanceIdMDC();
- }
- }
- log.info("Worker after kill all cache task, task size: {}, killed number: {}", workerTaskExecutors.size(),
- killNumber);
- }
}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/PhysicalTaskConfig.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/PhysicalTaskConfig.java
new file mode 100644
index 0000000000..2b5865c0e8
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/PhysicalTaskConfig.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.worker.config;
+
+import lombok.Data;
+
+@Data
+public class PhysicalTaskConfig {
+
+ private int taskExecutorThreadSize = Runtime.getRuntime().availableProcessors() * 2 + 1;
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java
index e391916a09..b092199c71 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java
@@ -41,7 +41,6 @@ import org.springframework.validation.annotation.Validated;
public class WorkerConfig implements Validator {
private int listenPort = 1234;
- private int execThreads = 10;
private Duration maxHeartbeatInterval = Duration.ofSeconds(10);
private int hostWeight = 100;
private WorkerServerLoadProtection serverLoadProtection = new WorkerServerLoadProtection();
@@ -52,10 +51,10 @@ public class WorkerConfig implements Validator {
private String workerAddress;
private String workerRegistryPath;
- private TaskExecuteThreadsFullPolicy taskExecuteThreadsFullPolicy = TaskExecuteThreadsFullPolicy.REJECT;
-
private TenantConfig tenantConfig = new TenantConfig();
+ private PhysicalTaskConfig physicalTaskConfig = new PhysicalTaskConfig();
+
@Override
public boolean supports(Class> clazz) {
return WorkerConfig.class.isAssignableFrom(clazz);
@@ -64,9 +63,6 @@ public class WorkerConfig implements Validator {
@Override
public void validate(Object target, Errors errors) {
WorkerConfig workerConfig = (WorkerConfig) target;
- if (workerConfig.getExecThreads() <= 0) {
- errors.rejectValue("exec-threads", null, "should be a positive value");
- }
if (workerConfig.getMaxHeartbeatInterval().getSeconds() <= 0) {
errors.rejectValue("max-heartbeat-interval", null, "shoule be a valid duration");
}
@@ -83,14 +79,13 @@ public class WorkerConfig implements Validator {
String config =
"\n****************************Worker Configuration**************************************" +
"\n listen-port -> " + listenPort +
- "\n exec-threads -> " + execThreads +
"\n max-heartbeat-interval -> " + maxHeartbeatInterval +
"\n host-weight -> " + hostWeight +
"\n tenantConfig -> " + tenantConfig +
"\n server-load-protection -> " + serverLoadProtection +
- "\n task-execute-threads-full-policy: " + taskExecuteThreadsFullPolicy +
"\n address -> " + workerAddress +
"\n registry-path: " + workerRegistryPath +
+ "\n physical-task-config -> " + physicalTaskConfig +
"\n****************************Worker Configuration**************************************";
log.info(config);
}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineDelegator.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineDelegator.java
new file mode 100644
index 0000000000..be70d0893d
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineDelegator.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.worker.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 org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterRequest;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.stereotype.Component;
+
+@Slf4j
+@Component
+public class PhysicalTaskEngineDelegator implements AutoCloseable {
+
+ private final TaskEngine taskEngine;
+
+ private final PhysicalTaskExecutorFactory physicalTaskExecutorFactory;
+
+ private final PhysicalTaskExecutorLifecycleEventReporter physicalTaskExecutorEventReporter;
+
+ private final PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
+
+ public PhysicalTaskEngineDelegator(final PhysicalTaskEngineFactory physicalTaskEngineFactory,
+ final PhysicalTaskExecutorFactory physicalTaskExecutorFactory,
+ final PhysicalTaskExecutorRepository physicalTaskExecutorRepository,
+ final PhysicalTaskExecutorLifecycleEventReporter physicalTaskExecutorEventReporter) {
+ this.physicalTaskExecutorFactory = physicalTaskExecutorFactory;
+ this.taskEngine = physicalTaskEngineFactory.createTaskEngine();
+ this.physicalTaskExecutorRepository = physicalTaskExecutorRepository;
+ this.physicalTaskExecutorEventReporter = physicalTaskExecutorEventReporter;
+ }
+
+ public void start() {
+ taskEngine.start();
+ physicalTaskExecutorEventReporter.start();
+ log.info("PhysicalTaskEngineDelegator started");
+ }
+
+ public void dispatchLogicTask(final TaskExecutionContext taskExecutionContext) {
+ final ITaskExecutor taskExecutor = physicalTaskExecutorFactory.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 ackPhysicalTaskExecutorLifecycleEventACK(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
+ physicalTaskExecutorEventReporter.receiveTaskExecutorLifecycleEventACK(taskExecutorLifecycleEventAck);
+ }
+
+ public boolean reassignWorkflowInstanceHost(final TaskExecutorReassignMasterRequest taskExecutorReassignMasterRequest) {
+ final int taskInstanceId = taskExecutorReassignMasterRequest.getTaskInstanceId();
+ final String workflowHost = taskExecutorReassignMasterRequest.getWorkflowHost();
+ // todo: Is this reassign can make sure there is no concurrent problem?
+ physicalTaskExecutorRepository.get(taskInstanceId).ifPresent(
+ taskExecutor -> taskExecutor.getTaskExecutionContext().setWorkflowInstanceHost(workflowHost));
+ return physicalTaskExecutorEventReporter.reassignWorkflowInstanceHost(taskInstanceId, workflowHost);
+ }
+
+ @Override
+ public void close() {
+ taskEngine.close();
+ }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java
new file mode 100644
index 0000000000..5d148aecde
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.worker.executor;
+
+import org.apache.dolphinscheduler.task.executor.TaskEngine;
+import org.apache.dolphinscheduler.task.executor.TaskEngineBuilder;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskEngineFactory {
+
+ @Autowired
+ private PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
+
+ @Autowired
+ private PhysicalTaskExecutorContainerProvider physicalTaskExecutorContainerDelegator;
+
+ @Autowired
+ private PhysicalTaskExecutorEventBusCoordinator physicalTaskExecutorEventBusCoordinator;
+
+ public TaskEngine createTaskEngine() {
+ final TaskEngineBuilder taskEngineBuilder = TaskEngineBuilder.builder()
+ .engineName("PhysicalTaskEngine")
+ .taskExecutorRepository(physicalTaskExecutorRepository)
+ .taskExecutorContainerDelegator(physicalTaskExecutorContainerDelegator)
+ .taskExecutorEventBusCoordinator(physicalTaskExecutorEventBusCoordinator)
+ .build();
+ return new TaskEngine(taskEngineBuilder);
+ }
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutor.java
new file mode 100644
index 0000000000..3886dad3fe
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutor.java
@@ -0,0 +1,142 @@
+/*
+ * 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.worker.executor;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.ApplicationInfo;
+import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
+import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+import org.apache.dolphinscheduler.server.worker.utils.TaskExecutionContextUtils;
+import org.apache.dolphinscheduler.server.worker.utils.TaskFilesTransferUtils;
+import org.apache.dolphinscheduler.server.worker.utils.TenantUtils;
+import org.apache.dolphinscheduler.task.executor.AbstractTaskExecutor;
+import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorState;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorStateMappings;
+import org.apache.dolphinscheduler.task.executor.events.TaskExecutorRuntimeContextChangedLifecycleEvent;
+
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class PhysicalTaskExecutor extends AbstractTaskExecutor {
+
+ private final WorkerConfig workerConfig;
+
+ private final StorageOperator storageOperator;
+
+ @Getter
+ private AbstractTask physicalTask;
+
+ private final PhysicalTaskPluginFactory physicalTaskPluginFactory;
+
+ public PhysicalTaskExecutor(final PhysicalTaskExecutorBuilder physicalTaskExecutorBuilder) {
+ super(physicalTaskExecutorBuilder.getTaskExecutionContext(),
+ physicalTaskExecutorBuilder.getTaskExecutorEventBus());
+ this.workerConfig = physicalTaskExecutorBuilder.getWorkerConfig();
+ this.storageOperator = physicalTaskExecutorBuilder.getStorageOperator();
+ this.physicalTaskPluginFactory = physicalTaskExecutorBuilder.getPhysicalTaskPluginFactory();
+ }
+
+ @Override
+ protected void initializeTaskPlugin() {
+ this.physicalTask = physicalTaskPluginFactory.createPhysicalTask(this);
+ log.info("Initialized physicalTask: {} successfully", taskExecutionContext.getTaskType());
+
+ this.physicalTask.init();
+
+ this.physicalTask.getParameters().setVarPool(taskExecutionContext.getVarPool());
+ log.info("Set taskVarPool: {} successfully", taskExecutionContext.getVarPool());
+ }
+
+ @Override
+ protected void doTriggerTaskPlugin() {
+ final ITaskExecutor taskExecutor = this;
+ physicalTask.handle(new TaskCallBack() {
+
+ @Override
+ public void updateRemoteApplicationInfo(final int taskInstanceId, final ApplicationInfo applicationInfo) {
+ taskExecutionContext.setAppIds(applicationInfo.getAppIds());
+ taskExecutorEventBus.publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(taskExecutor));
+ }
+
+ @Override
+ public void updateTaskInstanceInfo(final int taskInstanceId) {
+ taskExecutorEventBus.publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(taskExecutor));
+ }
+ });
+ }
+
+ @Override
+ protected TaskExecutorState doTrackTaskPluginStatus() {
+ return TaskExecutorStateMappings.mapState(physicalTask.getExitStatus());
+ }
+
+ @Override
+ public void pause() {
+ log.warn("The physical doesn't support pause");
+ }
+
+ @Override
+ public void kill() {
+ if (physicalTask != null) {
+ physicalTask.cancel();
+ }
+ }
+
+ @Override
+ protected void initializeTaskContext() {
+ super.initializeTaskContext();
+
+ taskExecutionContext.setTaskAppId(String.valueOf(taskExecutionContext.getTaskInstanceId()));
+
+ taskExecutionContext.setTenantCode(TenantUtils.getOrCreateActualTenant(workerConfig, taskExecutionContext));
+ log.info("TenantCode: {} check successfully", taskExecutionContext.getTenantCode());
+
+ TaskExecutionContextUtils.createTaskInstanceWorkingDirectory(taskExecutionContext);
+ log.info("TaskInstance working directory: {} create successfully", taskExecutionContext.getExecutePath());
+
+ final ResourceContext resourceContext = TaskExecutionContextUtils.downloadResourcesIfNeeded(
+ physicalTaskPluginFactory.getTaskChannel(this),
+ storageOperator,
+ taskExecutionContext);
+ taskExecutionContext.setResourceContext(resourceContext);
+ log.info("Download resources successfully: \n{}", taskExecutionContext.getResourceContext());
+
+ // todo: remove this. The cache should be deprecated
+ TaskFilesTransferUtils.downloadUpstreamFiles(taskExecutionContext, storageOperator);
+ log.info("Download upstream files: {} successfully",
+ TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.IN));
+
+ log.info("End initialize task {}", JSONUtils.toPrettyJsonString(taskExecutionContext));
+ }
+
+ @Override
+ public String toString() {
+ return "PhysicalTaskExecutor{" +
+ "id=" + taskExecutionContext.getTaskInstanceId() +
+ ", name=" + taskExecutionContext.getTaskName() +
+ ", stat=" + taskExecutorState.get() +
+ '}';
+ }
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorBuilder.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorBuilder.java
new file mode 100644
index 0000000000..e80bf9fe69
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorBuilder.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.worker.executor;
+
+import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorEventBus;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+
+@Data
+@Builder
+@AllArgsConstructor
+public class PhysicalTaskExecutorBuilder {
+
+ private TaskExecutionContext taskExecutionContext;
+
+ private WorkerConfig workerConfig;
+
+ private StorageOperator storageOperator;
+
+ @Builder.Default
+ private TaskExecutorEventBus taskExecutorEventBus = new TaskExecutorEventBus();
+
+ private PhysicalTaskPluginFactory physicalTaskPluginFactory;
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorContainerProvider.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorContainerProvider.java
new file mode 100644
index 0000000000..80a4e5c9f2
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorContainerProvider.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.worker.executor;
+
+import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+import org.apache.dolphinscheduler.task.executor.container.ExclusiveThreadTaskExecutorContainer;
+import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer;
+import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerProvider;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerConfig;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorContainerProvider implements ITaskExecutorContainerProvider {
+
+ private final ITaskExecutorContainer taskExecutorContainer;
+
+ public PhysicalTaskExecutorContainerProvider(final WorkerConfig workerConfig) {
+ final TaskExecutorContainerConfig containerConfig = TaskExecutorContainerConfig.builder()
+ .containerName("exclusive-task-executor-container")
+ .taskExecutorThreadPoolSize(workerConfig.getPhysicalTaskConfig().getTaskExecutorThreadSize())
+ .build();
+ this.taskExecutorContainer = new ExclusiveThreadTaskExecutorContainer(containerConfig);
+ }
+
+ @Override
+ public ITaskExecutorContainer getExecutorContainer() {
+ return taskExecutorContainer;
+ }
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java
similarity index 51%
rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java
rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java
index 68ebfba298..c9fe0f4aa6 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java
@@ -15,28 +15,18 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.extract.master.transportor;
+package org.apache.dolphinscheduler.server.worker.executor;
-import lombok.AllArgsConstructor;
-import lombok.Data;
-import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorEventBusCoordinator;
-@Data
-@NoArgsConstructor
-@AllArgsConstructor
-public class LogicTaskDispatchResponse {
+import org.springframework.stereotype.Component;
- private Integer taskInstanceId;
+@Component
+public class PhysicalTaskExecutorEventBusCoordinator extends TaskExecutorEventBusCoordinator {
- private boolean dispatchSuccess;
-
- private String message;
-
- public static LogicTaskDispatchResponse success(Integer taskInstanceId) {
- return new LogicTaskDispatchResponse(taskInstanceId, true, "dispatch success");
- }
-
- public static LogicTaskDispatchResponse failed(Integer taskInstanceId, String message) {
- return new LogicTaskDispatchResponse(taskInstanceId, false, message);
+ public PhysicalTaskExecutorEventBusCoordinator(final PhysicalTaskExecutorRepository physicalTaskExecutorRepository,
+ final PhysicalTaskExecutorLifecycleEventListener physicalTaskExecutorLifecycleEventListener) {
+ super("PhysicalTaskExecutorEventBusCoordinator", physicalTaskExecutorRepository);
+ registerTaskExecutorLifecycleEventListener(physicalTaskExecutorLifecycleEventListener);
}
}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventRemoteReporterClient.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventRemoteReporterClient.java
new file mode 100644
index 0000000000..18efd8839e
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventRemoteReporterClient.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.worker.executor;
+
+import org.apache.dolphinscheduler.extract.master.TaskExecutorEventRemoteReporterClient;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorEventRemoteReporterClient extends TaskExecutorEventRemoteReporterClient {
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorFactory.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorFactory.java
new file mode 100644
index 0000000000..7330bddcc2
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorFactory.java
@@ -0,0 +1,66 @@
+/*
+ * 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.worker.executor;
+
+import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+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 PhysicalTaskExecutorFactory implements ITaskExecutorFactory {
+
+ private final WorkerConfig workerConfig;
+
+ private final PhysicalTaskPluginFactory physicalTaskPluginFactory;
+
+ private final StorageOperator storageOperator;
+
+ public PhysicalTaskExecutorFactory(final WorkerConfig workerConfig,
+ final PhysicalTaskPluginFactory physicalTaskPluginFactory,
+ final StorageOperator storageOperator) {
+ this.workerConfig = workerConfig;
+ this.physicalTaskPluginFactory = physicalTaskPluginFactory;
+ this.storageOperator = storageOperator;
+ }
+
+ @Override
+ public ITaskExecutor createTaskExecutor(final TaskExecutionContext taskExecutionContext) {
+ assemblyTaskLogPath(taskExecutionContext);
+
+ final PhysicalTaskExecutorBuilder physicalTaskExecutorBuilder = PhysicalTaskExecutorBuilder.builder()
+ .taskExecutionContext(taskExecutionContext)
+ .workerConfig(workerConfig)
+ .storageOperator(storageOperator)
+ .physicalTaskPluginFactory(physicalTaskPluginFactory)
+ .build();
+ return new PhysicalTaskExecutor(physicalTaskExecutorBuilder);
+ }
+
+ private void assemblyTaskLogPath(final TaskExecutionContext taskExecutionContext) {
+ taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext));
+ }
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventListener.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventListener.java
new file mode 100644
index 0000000000..0bf12235a0
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventListener.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.worker.executor;
+
+import org.apache.dolphinscheduler.task.executor.listener.TaskExecutorLifecycleEventListener;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorLifecycleEventListener extends TaskExecutorLifecycleEventListener {
+
+ public PhysicalTaskExecutorLifecycleEventListener(
+ final PhysicalTaskExecutorContainerProvider physicalTaskExecutorContainerDelegator,
+ final PhysicalTaskExecutorRepository physicalTaskExecutorRepository,
+ final PhysicalTaskExecutorLifecycleEventReporter physicalTaskExecutorEventReporter) {
+ super(
+ physicalTaskExecutorContainerDelegator,
+ physicalTaskExecutorRepository,
+ physicalTaskExecutorEventReporter);
+ }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventReporter.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventReporter.java
new file mode 100644
index 0000000000..3be993adbe
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorLifecycleEventReporter.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.worker.executor;
+
+import org.apache.dolphinscheduler.task.executor.eventbus.TaskExecutorLifecycleEventRemoteReporter;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorLifecycleEventReporter extends TaskExecutorLifecycleEventRemoteReporter {
+
+ public PhysicalTaskExecutorLifecycleEventReporter(
+ final PhysicalTaskExecutorEventRemoteReporterClient physicalTaskExecutorEventRemoteReporterClient) {
+ super("PhysicalTaskExecutorLifecycleEventReporter", physicalTaskExecutorEventRemoteReporterClient);
+ }
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseRequest.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java
similarity index 74%
rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseRequest.java
rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java
index 0ac51e0212..01a9f7e4d2 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseRequest.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java
@@ -15,17 +15,13 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.extract.master.transportor;
+package org.apache.dolphinscheduler.server.worker.executor;
-import lombok.AllArgsConstructor;
-import lombok.Data;
-import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorRepository;
-@Data
-@NoArgsConstructor
-@AllArgsConstructor
-public class LogicTaskPauseRequest {
+import org.springframework.stereotype.Component;
- private int taskInstanceId;
+@Component
+public class PhysicalTaskExecutorRepository extends TaskExecutorRepository {
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
similarity index 51%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java
rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
index 9943ee1b5b..b37fc9ecf7 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
@@ -15,35 +15,24 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.runner.dispatcher;
+package org.apache.dolphinscheduler.server.worker.executor;
-import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
-@Slf4j
@Component
-public class TaskDispatchFactory {
-
- @Autowired
- private MasterTaskDispatcher masterTaskDispatcher;
+public class PhysicalTaskPluginFactory {
- @Autowired
- private WorkerTaskDispatcher workerTaskDispatcher;
-
- public TaskDispatcher getTaskDispatcher(String taskType) {
- if (TaskTypeUtils.isLogicTask(taskType)) {
- return masterTaskDispatcher;
- }
- return workerTaskDispatcher;
+ public AbstractTask createPhysicalTask(final PhysicalTaskExecutor physicalTaskExecutor) {
+ TaskChannel taskChannel = getTaskChannel(physicalTaskExecutor);
+ return taskChannel.createTask(physicalTaskExecutor.getTaskExecutionContext());
}
- public TaskDispatcher getTaskDispatcher(TaskInstance taskInstance) {
- return getTaskDispatcher(taskInstance.getTaskType());
+ public TaskChannel getTaskChannel(final PhysicalTaskExecutor physicalTaskExecutor) {
+ final String taskType = physicalTaskExecutor.getTaskExecutionContext().getTaskType();
+ return TaskPluginManager.getTaskChannel(taskType);
}
-
}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/MessageRetryRunner.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/MessageRetryRunner.java
deleted file mode 100644
index a4312e3cdb..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/MessageRetryRunner.java
+++ /dev/null
@@ -1,196 +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.worker.message;
-
-import org.apache.dolphinscheduler.common.constants.Constants;
-import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
-import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.collections4.MapUtils;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import lombok.Data;
-import lombok.NonNull;
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.context.annotation.Lazy;
-import org.springframework.stereotype.Component;
-
-import com.google.common.base.Objects;
-
-@Component
-@Slf4j
-public class MessageRetryRunner extends BaseDaemonThread {
-
- protected MessageRetryRunner() {
- super("WorkerMessageRetryRunnerThread");
- }
-
- private static final long MESSAGE_RETRY_WINDOW = Duration.ofMinutes(5L).toMillis();
-
- @Lazy
- @Autowired
- private List messageSenders;
-
- private final Map> messageSenderMap =
- new HashMap<>();
-
- private final Map> needToRetryMessages = new ConcurrentHashMap<>();
-
- @Override
- public synchronized void start() {
- log.info("Message retry runner staring");
- messageSenders.forEach(messageSender -> {
- messageSenderMap.put(messageSender.getMessageType(), messageSender);
- log.info("Injected message sender: {}", messageSender.getClass().getSimpleName());
- });
- super.start();
- log.info("Message retry runner started");
- }
-
- public void addRetryMessage(int taskInstanceId, @NonNull ITaskExecutionEvent iTaskExecutionEvent) {
- needToRetryMessages.computeIfAbsent(taskInstanceId, k -> Collections.synchronizedList(new ArrayList<>()))
- .add(TaskInstanceMessage.of(taskInstanceId, iTaskExecutionEvent.getEventType(),
- iTaskExecutionEvent));
- }
-
- public void removeRetryMessage(int taskInstanceId,
- @NonNull ITaskExecutionEvent.TaskInstanceExecutionEventType eventType) {
- List taskInstanceMessages = needToRetryMessages.get(taskInstanceId);
- if (taskInstanceMessages != null) {
- taskInstanceMessages.remove(TaskInstanceMessage.of(taskInstanceId, eventType, null));
- }
- }
-
- public void removeRetryMessages(int taskInstanceId) {
- needToRetryMessages.remove(taskInstanceId);
- }
-
- public boolean updateMessageHost(int taskInstanceId, String messageReceiverHost) {
- List taskInstanceMessages = this.needToRetryMessages.get(taskInstanceId);
- if (CollectionUtils.isEmpty(taskInstanceMessages)) {
- return false;
- }
- taskInstanceMessages.forEach(taskInstanceMessage -> {
- taskInstanceMessage.getEvent().setWorkflowInstanceHost(messageReceiverHost);
- });
- return true;
- }
-
- public void run() {
- while (!ServerLifeCycleManager.isStopped()) {
- try {
- if (MapUtils.isEmpty(needToRetryMessages)) {
- Thread.sleep(MESSAGE_RETRY_WINDOW);
- }
-
- long now = System.currentTimeMillis();
- Iterator>> iterator =
- needToRetryMessages.entrySet().iterator();
- while (iterator.hasNext()) {
- Map.Entry> taskEntry = iterator.next();
- Integer taskInstanceId = taskEntry.getKey();
- List taskInstanceMessages = taskEntry.getValue();
- if (taskInstanceMessages.isEmpty()) {
- iterator.remove();
- continue;
- }
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- try {
- for (TaskInstanceMessage taskInstanceMessage : taskInstanceMessages) {
- ITaskExecutionEvent.TaskInstanceExecutionEventType eventType =
- taskInstanceMessage.getEventType();
- ITaskExecutionEvent event = taskInstanceMessage.getEvent();
- if (now - event.getEventSendTime() > MESSAGE_RETRY_WINDOW) {
- log.info("Begin retry send message to master, event: {}", event);
- event.setEventSendTime(now);
- messageSenderMap.get(eventType).sendEvent(event);
- log.info("Success send message to master, event: {}", event);
- }
- }
- } catch (Exception e) {
- log.warn("Retry send message to master error", e);
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
- Thread.sleep(Constants.SLEEP_TIME_MILLIS);
- } catch (InterruptedException instance) {
- log.warn("The message retry thread is interrupted, will break this loop", instance);
- Thread.currentThread().interrupt();
- break;
- } catch (Exception ex) {
- log.error("Retry send message failed, get an known exception.", ex);
- }
- }
- }
-
- public void clearMessage() {
- needToRetryMessages.clear();
- }
-
- /**
- * If two message has the same taskInstanceId and messageType they will be considered as the same message
- */
- @Data
- public static class TaskInstanceMessage {
-
- private long taskInstanceId;
- private ITaskExecutionEvent.TaskInstanceExecutionEventType eventType;
- private ITaskExecutionEvent event;
-
- public static TaskInstanceMessage of(long taskInstanceId,
- ITaskExecutionEvent.TaskInstanceExecutionEventType eventType,
- ITaskExecutionEvent event) {
- TaskInstanceMessage taskInstanceMessage = new TaskInstanceMessage();
- taskInstanceMessage.setTaskInstanceId(taskInstanceId);
- taskInstanceMessage.setEventType(eventType);
- taskInstanceMessage.setEvent(event);
- return taskInstanceMessage;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- TaskInstanceMessage that = (TaskInstanceMessage) o;
- return taskInstanceId == that.taskInstanceId && eventType == that.eventType;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(taskInstanceId, eventType);
- }
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionDispatchEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionDispatchEventSender.java
deleted file mode 100644
index 5b3d12f407..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionDispatchEventSender.java
+++ /dev/null
@@ -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.server.worker.message;
-
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskExecutionDispatchEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(TaskExecutionDispatchEvent taskExecutionDispatchEvent) {
- Clients
- .withService(ITaskExecutionEventListener.class)
- .withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost())
- .onTaskInstanceDispatched(taskExecutionDispatchEvent);
- }
-
- @Override
- public TaskExecutionDispatchEvent buildEvent(TaskExecutionContext taskExecutionContext) {
- return TaskExecutionDispatchEvent.builder()
- .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost())
- .taskInstanceHost(taskExecutionContext.getHost())
- .build();
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.DISPATCH;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionFailedEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionFailedEventSender.java
deleted file mode 100644
index c3fa13550d..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionFailedEventSender.java
+++ /dev/null
@@ -1,56 +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.worker.message;
-
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskExecutionFailedEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(TaskExecutionFailedEvent message) {
- Clients
- .withService(ITaskExecutionEventListener.class)
- .withHost(message.getWorkflowInstanceHost())
- .onTaskInstanceExecutionFailed(message);
- }
-
- @Override
- public TaskExecutionFailedEvent buildEvent(TaskExecutionContext taskExecutionContext) {
- return TaskExecutionFailedEvent.builder()
- .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost())
- .taskInstanceHost(taskExecutionContext.getHost())
- .endTime(taskExecutionContext.getEndTime())
- .build();
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.FAILED;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionKilledEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionKilledEventSender.java
deleted file mode 100644
index e5222d8bc8..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionKilledEventSender.java
+++ /dev/null
@@ -1,56 +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.worker.message;
-
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskExecutionKilledEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(TaskExecutionKilledEvent message) {
- Clients
- .withService(ITaskExecutionEventListener.class)
- .withHost(message.getWorkflowInstanceHost())
- .onTaskInstanceExecutionKilled(message);
- }
-
- @Override
- public TaskExecutionKilledEvent buildEvent(TaskExecutionContext taskExecutionContext) {
- return TaskExecutionKilledEvent.builder()
- .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost())
- .taskInstanceHost(taskExecutionContext.getHost())
- .endTime(taskExecutionContext.getEndTime())
- .build();
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.KILLED;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionPausedEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionPausedEventSender.java
deleted file mode 100644
index 7e399542fb..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionPausedEventSender.java
+++ /dev/null
@@ -1,56 +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.worker.message;
-
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskExecutionPausedEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(TaskExecutionPausedEvent taskExecutionPausedEvent) {
- Clients
- .withService(ITaskExecutionEventListener.class)
- .withHost(taskExecutionPausedEvent.getWorkflowInstanceHost())
- .onTaskInstanceExecutionPaused(taskExecutionPausedEvent);
- }
-
- @Override
- public TaskExecutionPausedEvent buildEvent(TaskExecutionContext taskExecutionContext) {
- return TaskExecutionPausedEvent.builder()
- .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost())
- .taskInstanceHost(taskExecutionContext.getHost())
- .endTime(taskExecutionContext.getEndTime())
- .build();
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.PAUSED;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java
deleted file mode 100644
index 53581e7abd..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java
+++ /dev/null
@@ -1,61 +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.worker.message;
-
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import lombok.NonNull;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskInstanceExecutionRunningEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(final TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) {
- Clients.withService(ITaskExecutionEventListener.class)
- .withHost(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost())
- .onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);
- }
-
- @Override
- public TaskExecutionRunningEvent buildEvent(@NonNull TaskExecutionContext taskExecutionContext) {
- TaskExecutionRunningEvent taskExecuteRunningMessage = new TaskExecutionRunningEvent();
- taskExecuteRunningMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
- taskExecuteRunningMessage.setWorkflowInstanceId(taskExecutionContext.getWorkflowInstanceId());
- taskExecuteRunningMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus());
- taskExecuteRunningMessage.setLogPath(taskExecutionContext.getLogPath());
- taskExecuteRunningMessage.setWorkflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost());
- taskExecuteRunningMessage.setTaskInstanceHost(taskExecutionContext.getHost());
- taskExecuteRunningMessage.setStartTime(taskExecutionContext.getStartTime());
- taskExecuteRunningMessage.setExecutePath(taskExecutionContext.getExecutePath());
- taskExecuteRunningMessage.setAppIds(taskExecutionContext.getAppIds());
- return taskExecuteRunningMessage;
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.RUNNING;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java
index dd0375caf8..ca265274b7 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClient.java
@@ -32,7 +32,7 @@ import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.RegistryException;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskExecutorContainerProvider;
import org.apache.dolphinscheduler.server.worker.task.WorkerHeartBeatTask;
import org.apache.commons.collections4.CollectionUtils;
@@ -56,7 +56,7 @@ public class WorkerRegistryClient implements AutoCloseable {
private WorkerConfig workerConfig;
@Autowired
- private WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool;
+ private PhysicalTaskExecutorContainerProvider physicalTaskExecutorContainerDelegator;
@Autowired
private RegistryClient registryClient;
@@ -72,7 +72,7 @@ public class WorkerRegistryClient implements AutoCloseable {
workerConfig,
metricsProvider,
registryClient,
- workerTaskExecutorThreadPool);
+ physicalTaskExecutorContainerDelegator.getExecutorContainer());
}
public void start() {
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/PhysicalTaskExecutorOperatorImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/PhysicalTaskExecutorOperatorImpl.java
new file mode 100644
index 0000000000..e29ce197c0
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/PhysicalTaskExecutorOperatorImpl.java
@@ -0,0 +1,104 @@
+/*
+ * 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.worker.rpc;
+
+import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskEngineDelegator;
+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.exception.ExceptionUtils;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+@Slf4j
+@Component
+public class PhysicalTaskExecutorOperatorImpl implements IPhysicalTaskExecutorOperator {
+
+ @Autowired
+ private PhysicalTaskEngineDelegator physicalTaskEngineDelegator;
+
+ @Override
+ public TaskExecutorDispatchResponse dispatchTask(final TaskExecutorDispatchRequest taskExecutorDispatchRequest) {
+ log.info("Receive TaskExecutorDispatchResponse: {}", taskExecutorDispatchRequest);
+ final TaskExecutionContext taskExecutionContext = taskExecutorDispatchRequest.getTaskExecutionContext();
+ try {
+ physicalTaskEngineDelegator.dispatchLogicTask(taskExecutionContext);
+ log.info("Handle TaskExecutorDispatchResponse: {} success", taskExecutorDispatchRequest);
+ return TaskExecutorDispatchResponse.success();
+ } catch (Throwable throwable) {
+ log.error("Handle TaskExecutorDispatchResponse: {} failed", taskExecutorDispatchRequest, throwable);
+ return TaskExecutorDispatchResponse.failed(ExceptionUtils.getMessage(throwable));
+ }
+ }
+
+ @Override
+ public TaskExecutorKillResponse killTask(final TaskExecutorKillRequest taskExecutorKillRequest) {
+ log.info("Receive TaskExecutorKillRequest: {}", taskExecutorKillRequest);
+ final int taskInstanceId = taskExecutorKillRequest.getTaskInstanceId();
+ try {
+ physicalTaskEngineDelegator.killLogicTask(taskInstanceId);
+ log.info("Handle TaskExecutorKillRequest: {} success", taskExecutorKillRequest);
+ return TaskExecutorKillResponse.success();
+ } catch (Throwable throwable) {
+ log.error("Handle TaskExecutorKillRequest: {} failed", taskExecutorKillRequest, throwable);
+ return TaskExecutorKillResponse.fail(ExceptionUtils.getMessage(throwable));
+ }
+ }
+
+ @Override
+ public TaskExecutorPauseResponse pauseTask(final TaskExecutorPauseRequest taskPauseRequest) {
+ log.info("Receive TaskExecutorPauseRequest: {}", taskPauseRequest);
+ final int taskInstanceId = taskPauseRequest.getTaskInstanceId();
+ try {
+ physicalTaskEngineDelegator.pauseLogicTask(taskInstanceId);
+ log.info("Handle TaskExecutorPauseRequest: {} success", taskPauseRequest);
+ return TaskExecutorPauseResponse.success();
+ } catch (Throwable throwable) {
+ log.error("Handle TaskExecutorPauseRequest: {} failed", taskPauseRequest, throwable);
+ return TaskExecutorPauseResponse.fail(ExceptionUtils.getMessage(throwable));
+ }
+ }
+
+ @Override
+ public TaskExecutorReassignMasterResponse reassignWorkflowInstanceHost(final TaskExecutorReassignMasterRequest taskExecutorReassignMasterRequest) {
+ boolean success =
+ physicalTaskEngineDelegator.reassignWorkflowInstanceHost(taskExecutorReassignMasterRequest);
+ if (success) {
+ return TaskExecutorReassignMasterResponse.success();
+ }
+ return TaskExecutorReassignMasterResponse.failed("Reassign master host failed");
+ }
+
+ @Override
+ public void ackPhysicalTaskExecutorLifecycleEvent(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) {
+ log.info("Receive TaskExecutorLifecycleEventAck: {}", taskExecutorLifecycleEventAck);
+ physicalTaskEngineDelegator.ackPhysicalTaskExecutorLifecycleEventACK(taskExecutorLifecycleEventAck);
+ }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/StreamingTaskInstanceOperatorImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/StreamingTaskInstanceOperatorImpl.java
index 51b75aa457..f17b8882a3 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/StreamingTaskInstanceOperatorImpl.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/StreamingTaskInstanceOperatorImpl.java
@@ -23,9 +23,11 @@ import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceTrigge
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
import org.apache.dolphinscheduler.plugin.task.api.stream.StreamTask;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskExecutor;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskExecutorRepository;
+import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
+
+import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
@@ -37,7 +39,7 @@ import org.springframework.stereotype.Component;
public class StreamingTaskInstanceOperatorImpl implements IStreamingTaskInstanceOperator {
@Autowired
- private WorkerTaskExecutorThreadPool workerManager;
+ private PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
@Override
public TaskInstanceTriggerSavepointResponse triggerSavepoint(TaskInstanceTriggerSavepointRequest taskInstanceTriggerSavepointRequest) {
@@ -46,12 +48,13 @@ public class StreamingTaskInstanceOperatorImpl implements IStreamingTaskInstance
try {
int taskInstanceId = taskInstanceTriggerSavepointRequest.getTaskInstanceId();
LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(taskInstanceId);
- if (workerTaskExecutor == null) {
+ final Optional taskExecutorOptional = physicalTaskExecutorRepository.get(taskInstanceId);
+ if (!taskExecutorOptional.isPresent()) {
log.error("Cannot find WorkerTaskExecutor for taskInstance: {}", taskInstanceId);
return TaskInstanceTriggerSavepointResponse.fail("Cannot find TaskExecutionContext");
}
- AbstractTask task = workerTaskExecutor.getTask();
+ final PhysicalTaskExecutor taskExecutor = (PhysicalTaskExecutor) taskExecutorOptional.get();
+ AbstractTask task = taskExecutor.getPhysicalTask();
if (task == null) {
log.error("Cannot find StreamTask for taskInstance:{}", taskInstanceId);
return TaskInstanceTriggerSavepointResponse.fail("Cannot find StreamTask");
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceExecutionEventAckListenerImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceExecutionEventAckListenerImpl.java
deleted file mode 100644
index c086a77bfe..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceExecutionEventAckListenerImpl.java
+++ /dev/null
@@ -1,139 +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.worker.rpc;
-
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent.TaskInstanceExecutionEventType;
-import org.apache.dolphinscheduler.extract.worker.ITaskInstanceExecutionEventAckListener;
-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;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Slf4j
-@Component
-public class TaskInstanceExecutionEventAckListenerImpl implements ITaskInstanceExecutionEventAckListener {
-
- @Autowired
- private MessageRetryRunner messageRetryRunner;
-
- @Override
- public void handleTaskInstanceDispatchedEventAck(TaskInstanceExecutionDispatchedEventAck taskInstanceExecutionDispatchedEventAck) {
- try {
- final int taskInstanceId = taskInstanceExecutionDispatchedEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskInstanceDispatchedEventAck: {}", taskInstanceExecutionDispatchedEventAck);
- if (taskInstanceExecutionDispatchedEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.DISPATCH);
- } else {
- log.warn("TaskInstanceDispatchedEvent handle failed: {}", taskInstanceExecutionDispatchedEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
- @Override
- public void handleTaskInstanceExecutionRunningEventAck(TaskInstanceExecutionRunningEventAck taskInstanceExecutionRunningEventAck) {
- try {
- final int taskInstanceId = taskInstanceExecutionRunningEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskInstanceExecutionRunningEventAck: {}", taskInstanceExecutionRunningEventAck);
- if (taskInstanceExecutionRunningEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.RUNNING);
- } else {
- log.warn("TaskInstanceExecutionRunningEvent handle failed: {}", taskInstanceExecutionRunningEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
- @Override
- public void handleTaskExecutionSuccessEventAck(TaskExecutionSuccessEventAck taskExecutionSuccessEventAck) {
- try {
- final int taskInstanceId = taskExecutionSuccessEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskExecutionSuccessEventAck: {}", taskExecutionSuccessEventAck);
- if (taskExecutionSuccessEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.SUCCESS);
- } else {
- log.warn("TaskExecutionSuccessEvent handle failed: {}", taskExecutionSuccessEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
- @Override
- public void handleTaskExecutionPausedEventAck(TaskExecutionPausedEventAck taskExecutionPausedEventAck) {
- try {
- final int taskInstanceId = taskExecutionPausedEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskExecutionPausedEventAck: {}", taskExecutionPausedEventAck);
- if (taskExecutionPausedEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.PAUSED);
- } else {
- log.warn("TaskExecutionPausedEvent handle failed: {}", taskExecutionPausedEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
- @Override
- public void handleTaskExecutionFailedEventAck(TaskExecutionFailedEventAck taskExecutionFailedEventAck) {
- try {
- final int taskInstanceId = taskExecutionFailedEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskExecutionFailedEventAck: {}", taskExecutionFailedEventAck);
- if (taskExecutionFailedEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.FAILED);
- } else {
- log.warn("TaskExecutionFailedEvent handle failed: {}", taskExecutionFailedEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
- @Override
- public void handleTaskExecutionKilledEventAck(TaskExecutionKilledEventAck taskExecutionKilledEventAck) {
- try {
- final int taskInstanceId = taskExecutionKilledEventAck.getTaskInstanceId();
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Receive TaskExecutionKilledEventAck: {}", taskExecutionKilledEventAck);
- if (taskExecutionKilledEventAck.isSuccess()) {
- messageRetryRunner.removeRetryMessage(taskInstanceId, TaskInstanceExecutionEventType.KILLED);
- } else {
- log.warn("TaskExecutionKilledEvent handle failed: {}", taskExecutionKilledEventAck);
- }
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceOperatorImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceOperatorImpl.java
deleted file mode 100644
index d03df8a88c..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/TaskInstanceOperatorImpl.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.dolphinscheduler.server.worker.rpc;
-
-import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
-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;
-import org.apache.dolphinscheduler.server.worker.runner.operator.TaskInstanceOperationFunctionManager;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskInstanceOperatorImpl implements ITaskInstanceOperator {
-
- @Autowired
- private TaskInstanceOperationFunctionManager taskInstanceOperationFunctionManager;
-
- @Override
- public TaskInstanceDispatchResponse dispatchTask(final TaskInstanceDispatchRequest taskInstanceDispatchRequest) {
- return taskInstanceOperationFunctionManager.getTaskInstanceDispatchOperationFunction()
- .operate(taskInstanceDispatchRequest);
- }
-
- @Override
- public TaskInstanceKillResponse killTask(final TaskInstanceKillRequest taskInstanceKillRequest) {
- return taskInstanceOperationFunctionManager.getTaskInstanceKillOperationFunction()
- .operate(taskInstanceKillRequest);
- }
-
- @Override
- public TaskInstancePauseResponse pauseTask(final TaskInstancePauseRequest taskPauseRequest) {
- return taskInstanceOperationFunctionManager.getTaskInstancePauseOperationFunction()
- .operate(taskPauseRequest);
- }
-
- @Override
- public TakeOverTaskResponse takeOverTask(final TakeOverTaskRequest takeOverTaskRequest) {
- return taskInstanceOperationFunctionManager.getUpdateWorkflowHostOperationFunction()
- .operate(takeOverTaskRequest);
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerLogServiceImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerLogServiceImpl.java
index a8a0c5a2b7..b1bca6dfe6 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerLogServiceImpl.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerLogServiceImpl.java
@@ -17,21 +17,13 @@
package org.apache.dolphinscheduler.server.worker.rpc;
-import static org.apache.dolphinscheduler.common.constants.Constants.APPID_COLLECT;
-import static org.apache.dolphinscheduler.common.constants.Constants.DEFAULT_COLLECT_WAY;
-
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.LogUtils;
-import org.apache.dolphinscheduler.common.utils.PropertyUtils;
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;
import org.apache.dolphinscheduler.extract.common.transportor.TaskInstanceLogPageQueryResponse;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
import java.util.List;
@@ -62,21 +54,6 @@ public class WorkerLogServiceImpl implements ILogService {
return new TaskInstanceLogPageQueryResponse(logContent);
}
- @Override
- public GetAppIdResponse getAppId(GetAppIdRequest getAppIdRequest) {
- String appInfoPath = null;
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(getAppIdRequest.getTaskInstanceId());
- if (workerTaskExecutor != null) {
- // todo: remove this kind of logic, and remove get appId method, the appId should be send by worker rather
- // than query by master
- appInfoPath = workerTaskExecutor.getTaskExecutionContext().getAppInfoPath();
- }
- String logPath = getAppIdRequest.getLogPath();
- List appIds = org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils.getAppIds(logPath, appInfoPath,
- PropertyUtils.getString(APPID_COLLECT, DEFAULT_COLLECT_WAY));
- return new GetAppIdResponse(appIds);
- }
-
@Override
public void removeTaskInstanceLog(String taskInstanceLogAbsolutePath) {
FileUtils.deleteFile(taskInstanceLogAbsolutePath);
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerMessageSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerMessageSender.java
deleted file mode 100644
index 36c1345662..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerMessageSender.java
+++ /dev/null
@@ -1,73 +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.worker.rpc;
-
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
-import org.apache.dolphinscheduler.server.worker.message.TaskInstanceExecutionEventSender;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.PostConstruct;
-
-import lombok.NonNull;
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Component
-@Slf4j
-public class WorkerMessageSender {
-
- @Autowired
- private MessageRetryRunner messageRetryRunner;
-
- @Autowired
- private List messageSenders;
-
- private final Map messageSenderMap =
- new HashMap<>();
-
- @PostConstruct
- public void init() {
- messageSenders.forEach(messageSender -> messageSenderMap.put(messageSender.getMessageType(),
- messageSender));
- }
-
- // todo: use message rather than context
- public void sendMessageWithRetry(@NonNull TaskExecutionContext taskExecutionContext,
- @NonNull ITaskExecutionEvent.TaskInstanceExecutionEventType eventType) {
- TaskInstanceExecutionEventSender messageSender = messageSenderMap.get(eventType);
- if (messageSender == null) {
- log.error("The messageSender is invalidated, messageType: {}", eventType);
- return;
- }
- ITaskExecutionEvent iTaskExecutionEvent = messageSender.buildEvent(taskExecutionContext);
- try {
- messageRetryRunner.addRetryMessage(taskExecutionContext.getTaskInstanceId(), iTaskExecutionEvent);
- messageSender.sendEvent(iTaskExecutionEvent);
- } catch (Exception e) {
- log.error("Send message error, eventType: {}, event: {}", eventType, iTaskExecutionEvent);
- }
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutor.java
deleted file mode 100644
index f5abf9e577..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutor.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.dolphinscheduler.server.worker.runner;
-
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
-import org.apache.dolphinscheduler.plugin.task.api.TaskException;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import javax.annotation.Nullable;
-
-import lombok.NonNull;
-
-public class DefaultWorkerTaskExecutor extends WorkerTaskExecutor {
-
- public DefaultWorkerTaskExecutor(@NonNull TaskExecutionContext taskExecutionContext,
- @NonNull WorkerConfig workerConfig,
- @NonNull WorkerMessageSender workerMessageSender,
- @Nullable StorageOperator storageOperator,
- @NonNull WorkerRegistryClient workerRegistryClient) {
- super(taskExecutionContext,
- workerConfig,
- workerMessageSender,
- storageOperator,
- workerRegistryClient);
- }
-
- @Override
- public void executeTask(TaskCallBack taskCallBack) throws TaskException {
- if (task == null) {
- throw new IllegalArgumentException("The task plugin instance is not initialized");
- }
- task.handle(taskCallBack);
- }
-
- @Override
- protected void afterExecute() {
- super.afterExecute();
- }
-
- @Override
- protected void afterThrowing(Throwable throwable) throws TaskException {
- super.afterThrowing(throwable);
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorFactory.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorFactory.java
deleted file mode 100644
index 085deafb09..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorFactory.java
+++ /dev/null
@@ -1,61 +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.worker.runner;
-
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import javax.annotation.Nullable;
-
-import lombok.NonNull;
-
-public class DefaultWorkerTaskExecutorFactory
- implements
- WorkerTaskExecutorFactory {
-
- private final @NonNull TaskExecutionContext taskExecutionContext;
- private final @NonNull WorkerConfig workerConfig;
- private final @NonNull WorkerMessageSender workerMessageSender;
- private final @Nullable StorageOperator storageOperator;
- private final @NonNull WorkerRegistryClient workerRegistryClient;
-
- public DefaultWorkerTaskExecutorFactory(@NonNull TaskExecutionContext taskExecutionContext,
- @NonNull WorkerConfig workerConfig,
- @NonNull WorkerMessageSender workerMessageSender,
- @Nullable StorageOperator storageOperator,
- @NonNull WorkerRegistryClient workerRegistryClient) {
- this.taskExecutionContext = taskExecutionContext;
- this.workerConfig = workerConfig;
- this.workerMessageSender = workerMessageSender;
- this.storageOperator = storageOperator;
- this.workerRegistryClient = workerRegistryClient;
- }
-
- @Override
- public DefaultWorkerTaskExecutor createWorkerTaskExecutor() {
- return new DefaultWorkerTaskExecutor(
- taskExecutionContext,
- workerConfig,
- workerMessageSender,
- storageOperator,
- workerRegistryClient);
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskCallbackImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskCallbackImpl.java
deleted file mode 100644
index c05fab390b..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskCallbackImpl.java
+++ /dev/null
@@ -1,56 +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.worker.runner;
-
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.model.ApplicationInfo;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import lombok.Builder;
-import lombok.extern.slf4j.Slf4j;
-
-@Slf4j
-@Builder
-public class TaskCallbackImpl implements TaskCallBack {
-
- private final WorkerMessageSender workerMessageSender;
-
- private final TaskExecutionContext taskExecutionContext;
-
- public TaskCallbackImpl(WorkerMessageSender workerMessageSender, TaskExecutionContext taskExecutionContext) {
- this.workerMessageSender = workerMessageSender;
- this.taskExecutionContext = taskExecutionContext;
- }
-
- @Override
- public void updateRemoteApplicationInfo(int taskInstanceId, ApplicationInfo applicationInfo) {
- // todo: use listener
- taskExecutionContext.setAppIds(applicationInfo.getAppIds());
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.RUNNING);
- }
-
- @Override
- public void updateTaskInstanceInfo(int taskInstanceId) {
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.RUNNING);
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutor.java
deleted file mode 100644
index d519b1420e..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutor.java
+++ /dev/null
@@ -1,373 +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.worker.runner;
-
-import static ch.qos.logback.classic.ClassicConstants.FINALIZE_SESSION_MARKER;
-import static org.apache.dolphinscheduler.common.constants.Constants.DRY_RUN_FLAG_YES;
-import static org.apache.dolphinscheduler.common.constants.Constants.K8S_CONFIG_REGEX;
-import static org.apache.dolphinscheduler.common.constants.Constants.SINGLE_SLASH;
-
-import org.apache.dolphinscheduler.common.enums.WarningType;
-import org.apache.dolphinscheduler.common.log.SensitiveDataConverter;
-import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils;
-import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.extract.alert.IAlertOperator;
-import org.apache.dolphinscheduler.extract.alert.request.AlertSendRequest;
-import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse;
-import org.apache.dolphinscheduler.extract.base.client.Clients;
-import org.apache.dolphinscheduler.extract.base.utils.Host;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.plugin.datasource.api.utils.CommonUtils;
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
-import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
-import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
-import org.apache.dolphinscheduler.plugin.task.api.TaskException;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.TaskPluginException;
-import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
-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.log.TaskInstanceLogHeader;
-import org.apache.dolphinscheduler.plugin.task.api.model.TaskAlertInfo;
-import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-import org.apache.dolphinscheduler.server.worker.utils.TaskExecutionContextUtils;
-import org.apache.dolphinscheduler.server.worker.utils.TaskFilesTransferUtils;
-import org.apache.dolphinscheduler.server.worker.utils.TenantUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.NoSuchFileException;
-import java.util.Optional;
-
-import javax.annotation.Nullable;
-
-import lombok.NonNull;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Strings;
-
-public abstract class WorkerTaskExecutor implements Runnable {
-
- protected static final Logger log = LoggerFactory.getLogger(WorkerTaskExecutor.class);
-
- protected final TaskExecutionContext taskExecutionContext;
- protected final WorkerConfig workerConfig;
- protected final WorkerMessageSender workerMessageSender;
- protected final @Nullable StorageOperator storageOperator;
- protected final WorkerRegistryClient workerRegistryClient;
-
- protected @Nullable AbstractTask task;
-
- protected WorkerTaskExecutor(
- @NonNull TaskExecutionContext taskExecutionContext,
- @NonNull WorkerConfig workerConfig,
- @NonNull WorkerMessageSender workerMessageSender,
- @Nullable StorageOperator storageOperator,
- @NonNull WorkerRegistryClient workerRegistryClient) {
- this.taskExecutionContext = taskExecutionContext;
- this.workerConfig = workerConfig;
- this.workerMessageSender = workerMessageSender;
- this.storageOperator = storageOperator;
- this.workerRegistryClient = workerRegistryClient;
- SensitiveDataConverter.addMaskPattern(K8S_CONFIG_REGEX);
- }
-
- protected abstract void executeTask(TaskCallBack taskCallBack);
-
- protected void afterExecute() throws TaskException {
- if (task == null) {
- throw new TaskException("The current task instance is null");
- }
- sendAlertIfNeeded();
-
- sendTaskResult();
-
- WorkerTaskExecutorHolder.remove(taskExecutionContext.getTaskInstanceId());
- log.info("Remove the current task execute context from worker cache");
- clearTaskExecPathIfNeeded();
-
- }
-
- protected void afterThrowing(Throwable throwable) throws TaskException {
- if (cancelTask()) {
- log.info("Cancel the task successfully");
- }
- WorkerTaskExecutorHolder.remove(taskExecutionContext.getTaskInstanceId());
- taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE);
- taskExecutionContext.setEndTime(System.currentTimeMillis());
- workerMessageSender.sendMessageWithRetry(
- taskExecutionContext, ITaskExecutionEvent.TaskInstanceExecutionEventType.FAILED);
- log.info("Get a exception when execute the task, will send the task status: {} to master: {}",
- TaskExecutionStatus.FAILURE.name(), taskExecutionContext.getHost());
-
- }
-
- protected boolean cancelTask() {
- // cancel the task
- if (task == null) {
- return true;
- }
- try {
- task.cancel();
- ProcessUtils.cancelApplication(taskExecutionContext);
- return true;
- } catch (Exception e) {
- log.error("Cancel task failed, this will not affect the taskInstance status, but you need to check manual",
- e);
- return false;
- }
- }
-
- @Override
- public void run() {
- try {
- LogUtils.setWorkflowAndTaskInstanceIDMDC(taskExecutionContext.getWorkflowInstanceId(),
- taskExecutionContext.getTaskInstanceId());
- LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath());
-
- TaskInstanceLogHeader.printInitializeTaskContextHeader();
- initializeTask();
-
- if (DRY_RUN_FLAG_YES == taskExecutionContext.getDryRun()) {
- taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
- taskExecutionContext.setEndTime(System.currentTimeMillis());
- WorkerTaskExecutorHolder.remove(taskExecutionContext.getTaskInstanceId());
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.SUCCESS);
- log.info(
- "The current execute mode is dry run, will stop the subsequent process and set the taskInstance status to success");
- return;
- }
- TaskInstanceLogHeader.printLoadTaskInstancePluginHeader();
- beforeExecute();
-
- TaskCallBack taskCallBack = TaskCallbackImpl.builder()
- .workerMessageSender(workerMessageSender)
- .taskExecutionContext(taskExecutionContext)
- .build();
-
- TaskInstanceLogHeader.printExecuteTaskHeader();
- executeTask(taskCallBack);
-
- TaskInstanceLogHeader.printFinalizeTaskHeader();
- afterExecute();
- closeLogAppender();
- } catch (Throwable ex) {
- log.error("Task execute failed, due to meet an exception", ex);
- afterThrowing(ex);
- closeLogAppender();
- } finally {
- LogUtils.removeWorkflowAndTaskInstanceIdMDC();
- LogUtils.removeTaskInstanceLogFullPathMDC();
- }
- }
-
- protected void initializeTask() {
- log.info("Begin to initialize task");
-
- long taskStartTime = System.currentTimeMillis();
- taskExecutionContext.setStartTime(taskStartTime);
- log.info("Set task startTime: {}", taskStartTime);
-
- String taskAppId = String.format("%s_%s", taskExecutionContext.getWorkflowInstanceId(),
- taskExecutionContext.getTaskInstanceId());
- taskExecutionContext.setTaskAppId(taskAppId);
- log.info("Set task appId: {}", taskAppId);
-
- log.info("End initialize task {}", JSONUtils.toPrettyJsonString(taskExecutionContext));
- }
-
- protected void beforeExecute() {
- taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION);
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.RUNNING);
- log.info("Send task status {} master: {}", TaskExecutionStatus.RUNNING_EXECUTION.name(),
- taskExecutionContext.getHost());
-
- // In most of case the origin tenant is the same as the current tenant
- // Except `default` tenant. The originTenant is used to download the resources
- String originTenant = taskExecutionContext.getTenantCode();
- taskExecutionContext.setTenantCode(TenantUtils.getOrCreateActualTenant(workerConfig, taskExecutionContext));
- log.info("TenantCode: {} check successfully", taskExecutionContext.getTenantCode());
-
- TaskExecutionContextUtils.createTaskInstanceWorkingDirectory(taskExecutionContext);
- log.info("WorkflowInstanceExecDir: {} check successfully", taskExecutionContext.getExecutePath());
-
- TaskChannel taskChannel =
- Optional.ofNullable(TaskPluginManager.getTaskChannel(taskExecutionContext.getTaskType()))
- .orElseThrow(() -> new TaskPluginException(taskExecutionContext.getTaskType()
- + " task plugin not found, please check the task type is correct."));
-
- log.info("Create TaskChannel: {} successfully", taskChannel.getClass().getName());
-
- ResourceContext resourceContext = TaskExecutionContextUtils.downloadResourcesIfNeeded(taskChannel,
- storageOperator, taskExecutionContext);
- taskExecutionContext.setResourceContext(resourceContext);
- log.info("Download resources successfully: \n{}", taskExecutionContext.getResourceContext());
-
- TaskFilesTransferUtils.downloadUpstreamFiles(taskExecutionContext, storageOperator);
- log.info("Download upstream files: {} successfully",
- TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.IN));
-
- task = taskChannel.createTask(taskExecutionContext);
- log.info("Task plugin instance: {} create successfully", taskExecutionContext.getTaskType());
-
- // todo: remove the init method, this should initialize in constructor method
- task.init();
- log.info("Success initialized task plugin instance successfully");
-
- task.getParameters().setVarPool(taskExecutionContext.getVarPool());
- log.info("Set taskVarPool: {} successfully", taskExecutionContext.getVarPool());
-
- }
-
- protected void sendAlertIfNeeded() {
- if (!task.getNeedAlert()) {
- return;
- }
-
- // todo: We need to send the alert to the master rather than directly send to the alert server
- Optional alertServerAddressOptional = workerRegistryClient.getAlertServerAddress();
- if (!alertServerAddressOptional.isPresent()) {
- log.error("Cannot get alert server address, please check the alert server is running");
- return;
- }
- Host alertServerAddress = alertServerAddressOptional.get();
-
- TaskAlertInfo taskAlertInfo = task.getTaskAlertInfo();
- AlertSendRequest alertSendRequest = new AlertSendRequest(
- taskAlertInfo.getAlertGroupId(),
- taskAlertInfo.getTitle(),
- taskAlertInfo.getContent(),
- task.getExitStatus() == TaskExecutionStatus.SUCCESS ? WarningType.SUCCESS.getCode()
- : WarningType.FAILURE.getCode());
- try {
- final AlertSendResponse alertSendResponse = Clients
- .withService(IAlertOperator.class)
- .withHost(alertServerAddress.getAddress())
- .sendAlert(alertSendRequest);
- log.info("Send alert to: {} successfully, response: {}", alertServerAddress, alertSendResponse);
- } catch (Exception e) {
- log.error("Send alert: {} to: {} failed", alertSendRequest, alertServerAddress, e);
- }
- }
-
- protected void sendTaskResult() {
- taskExecutionContext.setCurrentExecutionStatus(task.getExitStatus());
- taskExecutionContext.setProcessId(task.getProcessId());
- taskExecutionContext.setAppIds(task.getAppIds());
- taskExecutionContext.setVarPool(JSONUtils.toJsonString(task.getParameters().getVarPool()));
- taskExecutionContext.setEndTime(System.currentTimeMillis());
-
- // upload out files and modify the "OUT FILE" property in VarPool
- TaskFilesTransferUtils.uploadOutputFiles(taskExecutionContext, storageOperator);
-
- log.info("Upload output files: {} successfully",
- TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.OUT));
-
- switch (taskExecutionContext.getCurrentExecutionStatus()) {
- case SUCCESS:
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.SUCCESS);
- break;
- case FAILURE:
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.FAILED);
- break;
- case PAUSE:
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.PAUSED);
- break;
- case KILL:
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.KILLED);
- break;
- default:
- workerMessageSender.sendMessageWithRetry(taskExecutionContext,
- ITaskExecutionEvent.TaskInstanceExecutionEventType.FAILED);
- break;
- }
- log.info("Send task execute status: {} to master : {}", taskExecutionContext.getCurrentExecutionStatus().name(),
- taskExecutionContext.getHost());
- }
-
- protected void clearTaskExecPathIfNeeded() {
- String execLocalPath = taskExecutionContext.getExecutePath();
- if (!CommonUtils.isDevelopMode()) {
- log.info("The current execute mode isn't develop mode, will clear the task execute file: {}",
- execLocalPath);
- // get exec dir
- if (Strings.isNullOrEmpty(execLocalPath)) {
- log.warn("The task execute file is {} no need to clear", taskExecutionContext.getTaskName());
- return;
- }
-
- if (SINGLE_SLASH.equals(execLocalPath)) {
- log.warn("The task execute file is '/', direct deletion is not allowed");
- return;
- }
-
- try {
- org.apache.commons.io.FileUtils.deleteDirectory(new File(execLocalPath));
- log.info("Success clear the task execute file: {}", execLocalPath);
- } catch (IOException e) {
- if (e instanceof NoSuchFileException) {
- // this is expected
- } else {
- log.error(
- "Delete task execute file: {} failed, this will not affect the task status, but you need to clear this manually",
- execLocalPath, e);
- }
- }
- } else {
- log.info("The current execute mode is develop mode, will not clear the task execute file: {}",
- execLocalPath);
- }
- }
-
- protected void closeLogAppender() {
- try {
- if (RemoteLogUtils.isRemoteLoggingEnable()) {
- RemoteLogUtils.sendRemoteLog(taskExecutionContext.getLogPath());
- log.info("Log handler sends task log {} to remote storage asynchronously.",
- taskExecutionContext.getLogPath());
- }
- } catch (Exception ex) {
- log.error("Send remote log failed", ex);
- } finally {
- log.info(FINALIZE_SESSION_MARKER, FINALIZE_SESSION_MARKER.toString());
- }
- }
-
- public @NonNull TaskExecutionContext getTaskExecutionContext() {
- return taskExecutionContext;
- }
-
- public @Nullable AbstractTask getTask() {
- return task;
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactoryBuilder.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactoryBuilder.java
deleted file mode 100644
index 4cb1739abc..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactoryBuilder.java
+++ /dev/null
@@ -1,64 +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.worker.runner;
-
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Component
-public class WorkerTaskExecutorFactoryBuilder {
-
- @Autowired
- private WorkerConfig workerConfig;
-
- @Autowired
- private WorkerMessageSender workerMessageSender;
-
- @Autowired(required = false)
- private StorageOperator storageOperator;
-
- @Autowired
- private WorkerRegistryClient workerRegistryClient;
-
- public WorkerTaskExecutorFactoryBuilder(
- WorkerConfig workerConfig,
- WorkerMessageSender workerMessageSender,
- WorkerTaskExecutorThreadPool workerManager,
- StorageOperator storageOperator,
- WorkerRegistryClient workerRegistryClient) {
- this.workerConfig = workerConfig;
- this.workerMessageSender = workerMessageSender;
- this.storageOperator = storageOperator;
- this.workerRegistryClient = workerRegistryClient;
- }
-
- public WorkerTaskExecutorFactory extends WorkerTaskExecutor> createWorkerTaskExecutorFactory(TaskExecutionContext taskExecutionContext) {
- return new DefaultWorkerTaskExecutorFactory(taskExecutionContext,
- workerConfig,
- workerMessageSender,
- storageOperator,
- workerRegistryClient);
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorHolder.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorHolder.java
deleted file mode 100644
index e44b23152c..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorHolder.java
+++ /dev/null
@@ -1,58 +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.worker.runner;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Used to store all running and waiting {@link WorkerTaskExecutor}. If the task has been finished, it will be removed from the map.
- */
-public class WorkerTaskExecutorHolder {
-
- private static final Map workerTaskExecutorMap = new HashMap<>();
-
- public static void put(WorkerTaskExecutor workerTaskExecutor) {
- int taskInstanceId = workerTaskExecutor.getTaskExecutionContext().getTaskInstanceId();
- if (workerTaskExecutorMap.containsKey(taskInstanceId)) {
- throw new IllegalArgumentException("TaskInstance: " + taskInstanceId + " already exists");
- }
- workerTaskExecutorMap.put(taskInstanceId, workerTaskExecutor);
- }
-
- public static WorkerTaskExecutor get(int taskInstanceId) {
- return workerTaskExecutorMap.get(taskInstanceId);
- }
-
- public static WorkerTaskExecutor remove(int taskInstanceId) {
- return workerTaskExecutorMap.remove(taskInstanceId);
- }
-
- public static void clear() {
- workerTaskExecutorMap.clear();
- }
-
- public static int size() {
- return workerTaskExecutorMap.size();
- }
-
- public static Collection getAllTaskExecutor() {
- return workerTaskExecutorMap.values();
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java
deleted file mode 100644
index d4b0206394..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPool.java
+++ /dev/null
@@ -1,113 +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.worker.runner;
-
-import org.apache.dolphinscheduler.common.thread.ThreadUtils;
-import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent;
-import org.apache.dolphinscheduler.server.worker.config.TaskExecuteThreadsFullPolicy;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import java.util.concurrent.ThreadPoolExecutor;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.stereotype.Component;
-
-@Component
-@Slf4j
-public class WorkerTaskExecutorThreadPool {
-
- private final WorkerMessageSender workerMessageSender;
-
- private final ThreadPoolExecutor threadPoolExecutor;
-
- private final WorkerConfig workerConfig;
-
- public WorkerTaskExecutorThreadPool(WorkerConfig workerConfig, WorkerMessageSender workerMessageSender) {
- this.threadPoolExecutor =
- ThreadUtils.newDaemonFixedThreadExecutor("WorkerTaskExecutorThreadPool", workerConfig.getExecThreads());
- threadPoolExecutor.prestartAllCoreThreads();
- this.workerConfig = workerConfig;
- this.workerMessageSender = workerMessageSender;
-
- WorkerServerMetrics.registerWorkerExecuteQueueSizeGauge(this::getWaitingTaskExecutorSize);
- WorkerServerMetrics.registerWorkerActiveExecuteThreadGauge(this::getRunningTaskExecutorSize);
- }
-
- public boolean submitWorkerTaskExecutor(WorkerTaskExecutor workerTaskExecutor) {
- synchronized (WorkerTaskExecutorThreadPool.class) {
- if (TaskExecuteThreadsFullPolicy.CONTINUE.equals(workerConfig.getTaskExecuteThreadsFullPolicy())) {
- WorkerTaskExecutorHolder.put(workerTaskExecutor);
- sendDispatchedEvent(workerTaskExecutor);
- threadPoolExecutor.execute(workerTaskExecutor);
- return true;
- }
- if (isOverload()) {
- log.warn("WorkerTaskExecutorThreadPool is overload, cannot submit new WorkerTaskExecutor");
- WorkerServerMetrics.incWorkerSubmitQueueIsFullCount();
- return false;
- }
- WorkerTaskExecutorHolder.put(workerTaskExecutor);
- sendDispatchedEvent(workerTaskExecutor);
- threadPoolExecutor.execute(workerTaskExecutor);
- return true;
- }
- }
-
- private void sendDispatchedEvent(WorkerTaskExecutor workerTaskExecutor) {
- workerMessageSender.sendMessageWithRetry(
- workerTaskExecutor.getTaskExecutionContext(),
- ITaskExecutionEvent.TaskInstanceExecutionEventType.DISPATCH);
- }
-
- public boolean isOverload() {
- return WorkerTaskExecutorHolder.size() >= workerConfig.getExecThreads();
- }
-
- public int getWaitingTaskExecutorSize() {
- if (WorkerTaskExecutorHolder.size() <= workerConfig.getExecThreads()) {
- return 0;
- } else {
- return WorkerTaskExecutorHolder.size() - workerConfig.getExecThreads();
- }
- }
-
- public int getRunningTaskExecutorSize() {
- return Math.min(WorkerTaskExecutorHolder.size(), workerConfig.getExecThreads());
- }
-
- public double getThreadPoolUsage() {
- return (double) WorkerTaskExecutorHolder.size() / workerConfig.getExecThreads();
- }
-
- /**
- * Kill tasks that have not been executed, e.g. waiting in the queue
- */
- public void killTaskBeforeExecuteByInstanceId(Integer taskInstanceId) {
- synchronized (WorkerTaskExecutorThreadPool.class) {
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(taskInstanceId);
- threadPoolExecutor.remove(workerTaskExecutor);
- }
- }
-
- public void clearTask() {
- threadPoolExecutor.getQueue().clear();
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TakeOverTaskOperationFunction.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TakeOverTaskOperationFunction.java
deleted file mode 100644
index bc5d612b66..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TakeOverTaskOperationFunction.java
+++ /dev/null
@@ -1,80 +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.worker.runner.operator;
-
-import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskRequest;
-import org.apache.dolphinscheduler.extract.worker.transportor.TakeOverTaskResponse;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Slf4j
-@Component
-public class TakeOverTaskOperationFunction
- implements
- ITaskInstanceOperationFunction {
-
- @Autowired
- private MessageRetryRunner messageRetryRunner;
-
- public TakeOverTaskOperationFunction(MessageRetryRunner messageRetryRunner) {
- this.messageRetryRunner = messageRetryRunner;
- }
-
- @Override
- public TakeOverTaskResponse operate(TakeOverTaskRequest takeOverTaskRequest) {
- try {
- final int taskInstanceId = takeOverTaskRequest.getTaskInstanceId();
- final String workflowHost = takeOverTaskRequest.getWorkflowHost();
-
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- log.info("Received TakeOverTaskRequest: {}", takeOverTaskRequest);
-
- boolean updateWorkerTaskExecutor = updateHostInWorkflowTaskExecutor(taskInstanceId, workflowHost);
- boolean updateMessage = updateHostInMessage(taskInstanceId, workflowHost);
- if (updateWorkerTaskExecutor || updateMessage) {
- return TakeOverTaskResponse.success();
- }
- return TakeOverTaskResponse.failed("The taskInstance is not in the worker");
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- LogUtils.removeTaskInstanceLogFullPathMDC();
- }
- }
-
- private boolean updateHostInWorkflowTaskExecutor(int taskInstanceId, String workflowHost) {
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(taskInstanceId);
- if (workerTaskExecutor == null) {
- return false;
- }
- TaskExecutionContext taskExecutionContext = workerTaskExecutor.getTaskExecutionContext();
- taskExecutionContext.setWorkflowInstanceHost(workflowHost);
- return true;
- }
-
- private boolean updateHostInMessage(int taskInstanceId, String workflowHost) {
- return messageRetryRunner.updateMessageHost(taskInstanceId, workflowHost);
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceDispatchOperationFunction.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceDispatchOperationFunction.java
deleted file mode 100644
index 4cc40ada6c..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceDispatchOperationFunction.java
+++ /dev/null
@@ -1,99 +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.worker.runner.operator;
-
-import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchRequest;
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceDispatchResponse;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.metrics.TaskMetrics;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorFactoryBuilder;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Slf4j
-@Component
-public class TaskInstanceDispatchOperationFunction
- implements
- ITaskInstanceOperationFunction {
-
- @Autowired
- private WorkerConfig workerConfig;
-
- @Autowired
- private WorkerTaskExecutorFactoryBuilder workerTaskExecutorFactoryBuilder;
-
- @Autowired
- private WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool;
-
- @Autowired
- private WorkerMessageSender workerMessageSender;
-
- public TaskInstanceDispatchOperationFunction(
- WorkerConfig workerConfig,
- WorkerTaskExecutorFactoryBuilder workerTaskExecutorFactoryBuilder,
- WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool) {
- this.workerConfig = workerConfig;
- this.workerTaskExecutorFactoryBuilder = workerTaskExecutorFactoryBuilder;
- this.workerTaskExecutorThreadPool = workerTaskExecutorThreadPool;
- }
-
- @Override
- public TaskInstanceDispatchResponse operate(TaskInstanceDispatchRequest taskInstanceDispatchRequest) {
- log.info("Receive TaskInstanceDispatchRequest: {}", taskInstanceDispatchRequest);
- TaskExecutionContext taskExecutionContext = taskInstanceDispatchRequest.getTaskExecutionContext();
- try {
- taskExecutionContext.setHost(workerConfig.getWorkerAddress());
- taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext));
-
- LogUtils.setWorkflowAndTaskInstanceIDMDC(taskExecutionContext.getWorkflowInstanceId(),
- taskExecutionContext.getTaskInstanceId());
-
- // check server status, if server is not running, return failed to reject this task
- if (!ServerLifeCycleManager.isRunning()) {
- log.error("server is not running. reject task: {}", taskExecutionContext.getWorkflowInstanceId());
- return TaskInstanceDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(),
- "server is not running");
- }
-
- TaskMetrics.incrTaskTypeExecuteCount(taskExecutionContext.getTaskType());
-
- WorkerTaskExecutor workerTaskExecutor = workerTaskExecutorFactoryBuilder
- .createWorkerTaskExecutorFactory(taskExecutionContext)
- .createWorkerTaskExecutor();
- if (!workerTaskExecutorThreadPool.submitWorkerTaskExecutor(workerTaskExecutor)) {
- log.info("Submit task: {} to wait queue failed", taskExecutionContext.getTaskName());
- return TaskInstanceDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(),
- "WorkerManagerThread is full");
- } else {
- log.info("Submit task: {} to wait queue success", taskExecutionContext.getTaskName());
- return TaskInstanceDispatchResponse.success(taskExecutionContext.getTaskInstanceId());
- }
- } finally {
- LogUtils.removeWorkflowAndTaskInstanceIdMDC();
- }
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceKillOperationFunction.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceKillOperationFunction.java
deleted file mode 100644
index fe114ae6c9..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceKillOperationFunction.java
+++ /dev/null
@@ -1,150 +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.worker.runner.operator;
-
-import org.apache.dolphinscheduler.common.utils.OSUtils;
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest;
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse;
-import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
-import org.apache.dolphinscheduler.plugin.task.api.TaskException;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-import com.google.common.base.Strings;
-
-@Slf4j
-@Component
-public class TaskInstanceKillOperationFunction
- implements
- ITaskInstanceOperationFunction {
-
- @Autowired
- private WorkerTaskExecutorThreadPool workerManager;
-
- @Autowired
- private MessageRetryRunner messageRetryRunner;
-
- public TaskInstanceKillOperationFunction(
- WorkerTaskExecutorThreadPool workerManager,
- MessageRetryRunner messageRetryRunner) {
- this.workerManager = workerManager;
- this.messageRetryRunner = messageRetryRunner;
- }
-
- @Override
- public TaskInstanceKillResponse operate(TaskInstanceKillRequest taskInstanceKillRequest) {
- log.info("Receive TaskInstanceKillRequest: {}", taskInstanceKillRequest);
-
- int taskInstanceId = taskInstanceKillRequest.getTaskInstanceId();
- try {
- LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(taskInstanceId);
- if (workerTaskExecutor == null) {
- log.error("Cannot find WorkerTaskExecutor for taskInstance: {}", taskInstanceId);
- return TaskInstanceKillResponse.fail("Cannot find WorkerTaskExecutor");
- }
- TaskExecutionContext taskExecutionContext = workerTaskExecutor.getTaskExecutionContext();
-
- LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath());
-
- boolean result = doKill(taskExecutionContext);
- this.cancelApplication(workerTaskExecutor);
-
- int processId = taskExecutionContext.getProcessId();
- if (processId == 0) {
- workerManager.killTaskBeforeExecuteByInstanceId(taskInstanceId);
- taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.KILL);
- // todo: the task might be executed, but the processId is 0
- WorkerTaskExecutorHolder.remove(taskInstanceId);
- log.info("The task has not been executed and has been cancelled, task id:{}", taskInstanceId);
- return TaskInstanceKillResponse.success(taskExecutionContext);
- }
-
- taskExecutionContext
- .setCurrentExecutionStatus(result ? TaskExecutionStatus.KILL : TaskExecutionStatus.FAILURE);
-
- WorkerTaskExecutorHolder.remove(taskInstanceId);
- messageRetryRunner.removeRetryMessages(taskInstanceId);
- return TaskInstanceKillResponse.success(taskExecutionContext);
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- LogUtils.removeTaskInstanceLogFullPathMDC();
- }
- }
-
- private boolean doKill(TaskExecutionContext taskExecutionContext) {
- // kill system process
- boolean processFlag = killProcess(taskExecutionContext.getTenantCode(), taskExecutionContext.getProcessId());
-
- // kill yarn or k8s application
- try {
- ProcessUtils.cancelApplication(taskExecutionContext);
- } catch (TaskException e) {
- return false;
- }
- return processFlag;
- }
-
- protected void cancelApplication(WorkerTaskExecutor workerTaskExecutor) {
- AbstractTask task = workerTaskExecutor.getTask();
- if (task == null) {
- log.warn("task not found, taskInstanceId: {}",
- workerTaskExecutor.getTaskExecutionContext().getTaskInstanceId());
- return;
- }
- try {
- task.cancel();
- } catch (Exception e) {
- log.error("kill task error", e);
- }
- log.info("kill task by cancelApplication, taskInstanceId: {}",
- workerTaskExecutor.getTaskExecutionContext().getTaskInstanceId());
- }
-
- protected boolean killProcess(String tenantCode, Integer processId) {
- // todo: directly interrupt the process
- boolean processFlag = true;
- if (processId == null || processId.equals(0)) {
- return true;
- }
- try {
- String pidsStr = ProcessUtils.getPidsStr(processId);
- if (!Strings.isNullOrEmpty(pidsStr)) {
- String cmd = String.format("kill -9 %s", pidsStr);
- cmd = OSUtils.getSudoCmd(tenantCode, cmd);
- log.info("process id:{}, cmd:{}", processId, cmd);
- OSUtils.exeCmd(cmd);
- }
- } catch (Exception e) {
- processFlag = false;
- log.error("kill task error", e);
- }
- return processFlag;
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionManager.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionManager.java
deleted file mode 100644
index a32ced01ab..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionManager.java
+++ /dev/null
@@ -1,65 +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.worker.runner.operator;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskInstanceOperationFunctionManager {
-
- @Autowired
- private TaskInstanceKillOperationFunction taskInstanceKillOperationFunction;
-
- @Autowired
- private TakeOverTaskOperationFunction takeOverTaskOperationFunction;
-
- @Autowired
- private TaskInstanceDispatchOperationFunction taskInstanceDispatchOperationFunction;
-
- @Autowired
- private TaskInstancePauseOperationFunction taskInstancePauseOperationFunction;
-
- public TaskInstanceOperationFunctionManager(
- TaskInstanceKillOperationFunction taskInstanceKillOperationFunction,
- TakeOverTaskOperationFunction takeOverTaskOperationFunction,
- TaskInstanceDispatchOperationFunction taskInstanceDispatchOperationFunction,
- TaskInstancePauseOperationFunction taskInstancePauseOperationFunction) {
- this.taskInstanceKillOperationFunction = taskInstanceKillOperationFunction;
- this.takeOverTaskOperationFunction = takeOverTaskOperationFunction;
- this.taskInstanceDispatchOperationFunction = taskInstanceDispatchOperationFunction;
- this.taskInstancePauseOperationFunction = taskInstancePauseOperationFunction;
- }
-
- public TaskInstanceKillOperationFunction getTaskInstanceKillOperationFunction() {
- return taskInstanceKillOperationFunction;
- }
-
- public TakeOverTaskOperationFunction getUpdateWorkflowHostOperationFunction() {
- return takeOverTaskOperationFunction;
- }
-
- public TaskInstanceDispatchOperationFunction getTaskInstanceDispatchOperationFunction() {
- return taskInstanceDispatchOperationFunction;
- }
-
- public TaskInstancePauseOperationFunction getTaskInstancePauseOperationFunction() {
- return taskInstancePauseOperationFunction;
- }
-
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstancePauseOperationFunction.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstancePauseOperationFunction.java
deleted file mode 100644
index 233f215e59..0000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstancePauseOperationFunction.java
+++ /dev/null
@@ -1,45 +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.worker.runner.operator;
-
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseRequest;
-import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseResponse;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.stereotype.Component;
-
-@Slf4j
-@Component
-public class TaskInstancePauseOperationFunction
- implements
- ITaskInstanceOperationFunction {
-
- @Override
- public TaskInstancePauseResponse operate(TaskInstancePauseRequest taskInstancePauseRequest) {
- try {
- LogUtils.setTaskInstanceIdMDC(taskInstancePauseRequest.getTaskInstanceId());
- log.info("Receive TaskInstancePauseRequest: {}", taskInstancePauseRequest);
- log.info("TaskInstancePauseOperationFunction is not support for worker task yet!");
- return TaskInstancePauseResponse.success();
- } finally {
- LogUtils.removeTaskInstanceIdMDC();
- }
- }
-}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
index a2e3f37825..f5bce3b766 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java
@@ -31,7 +31,7 @@ import org.apache.dolphinscheduler.registry.api.utils.RegistryUtils;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.dolphinscheduler.server.worker.config.WorkerServerLoadProtection;
import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
+import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@@ -43,26 +43,27 @@ public class WorkerHeartBeatTask extends BaseHeartBeatTask {
private final RegistryClient registryClient;
private final MetricsProvider metricsProvider;
- private final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool;
private final int processId;
+ private final ITaskExecutorContainer taskExecutorContainer;
+
public WorkerHeartBeatTask(@NonNull WorkerConfig workerConfig,
@NonNull MetricsProvider metricsProvider,
@NonNull RegistryClient registryClient,
- @NonNull WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool) {
+ @NonNull ITaskExecutorContainer taskExecutorContainer) {
super("WorkerHeartBeatTask", workerConfig.getMaxHeartbeatInterval().toMillis());
this.metricsProvider = metricsProvider;
this.workerConfig = workerConfig;
this.registryClient = registryClient;
- this.workerTaskExecutorThreadPool = workerTaskExecutorThreadPool;
+ this.taskExecutorContainer = taskExecutorContainer;
this.processId = OSUtils.getProcessID();
}
@Override
public WorkerHeartBeat getHeartBeat() {
SystemMetrics systemMetrics = metricsProvider.getSystemMetrics();
- ServerStatus serverStatus = getServerStatus(systemMetrics, workerConfig, workerTaskExecutorThreadPool);
+ ServerStatus serverStatus = getServerStatus(systemMetrics, workerConfig, taskExecutorContainer);
return WorkerHeartBeat.builder()
.startupTime(ServerLifeCycleManager.getServerStartupTime())
@@ -74,7 +75,7 @@ public class WorkerHeartBeatTask extends BaseHeartBeatTask {
.diskUsage(systemMetrics.getDiskUsedPercentage())
.processId(processId)
.workerHostWeight(workerConfig.getHostWeight())
- .threadPoolUsage(workerTaskExecutorThreadPool.getThreadPoolUsage())
+ .threadPoolUsage(taskExecutorContainer.slotUsage())
.serverStatus(serverStatus)
.host(NetUtils.getHost())
.port(workerConfig.getListenPort())
@@ -105,8 +106,8 @@ public class WorkerHeartBeatTask extends BaseHeartBeatTask {
private ServerStatus getServerStatus(SystemMetrics systemMetrics,
WorkerConfig workerConfig,
- WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool) {
- if (workerTaskExecutorThreadPool.isOverload()) {
+ ITaskExecutorContainer taskExecutorContainer) {
+ if (taskExecutorContainer.slotUsage() == 1) {
return ServerStatus.BUSY;
}
WorkerServerLoadProtection serverLoadProtection = workerConfig.getServerLoadProtection();
diff --git a/dolphinscheduler-worker/src/main/resources/application.yaml b/dolphinscheduler-worker/src/main/resources/application.yaml
index c4077e623d..d9da1ec244 100644
--- a/dolphinscheduler-worker/src/main/resources/application.yaml
+++ b/dolphinscheduler-worker/src/main/resources/application.yaml
@@ -42,8 +42,6 @@ registry:
worker:
# worker listener port
listen-port: 1234
- # worker execute thread number to limit task instances in parallel
- exec-threads: 100
# worker heartbeat interval
max-heartbeat-interval: 10s
# worker host weight to dispatch tasks, default value 100
@@ -60,6 +58,9 @@ worker:
# Worker max disk usage , when the worker's disk usage is smaller then this value, worker server can be dispatched tasks.
max-disk-usage-percentage-thresholds: 0.7
task-execute-threads-full-policy: REJECT
+ physical-task-config:
+ # The number of threads in the Physical task engine that used to execute tasks
+ task-executor-thread-size: 100
tenant-config:
# 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.
auto-create-tenant-enabled: true
diff --git a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClientTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClientTest.java
index 54f9033e4e..636ba2e461 100644
--- a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClientTest.java
+++ b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryClientTest.java
@@ -28,7 +28,9 @@ import org.apache.dolphinscheduler.registry.api.RegistryClient;
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.dolphinscheduler.server.worker.config.WorkerServerLoadProtection;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
+import org.apache.dolphinscheduler.server.worker.executor.PhysicalTaskExecutorContainerProvider;
+import org.apache.dolphinscheduler.task.executor.container.ExclusiveThreadTaskExecutorContainer;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerConfig;
import java.time.Duration;
import java.util.ArrayList;
@@ -42,8 +44,6 @@ import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* worker registry test
@@ -51,7 +51,6 @@ import org.slf4j.LoggerFactory;
@ExtendWith(MockitoExtension.class)
public class WorkerRegistryClientTest {
- private static final Logger log = LoggerFactory.getLogger(WorkerRegistryClientTest.class);
@InjectMocks
private WorkerRegistryClient workerRegistryClient;
@Mock
@@ -61,13 +60,21 @@ public class WorkerRegistryClientTest {
@Mock
private MetricsProvider metricsProvider;
@Mock
- private WorkerTaskExecutorThreadPool workerManagerThread;
+ private PhysicalTaskExecutorContainerProvider physicalTaskExecutorContainerDelegator;
@Mock
private IStoppable stoppable;
@Test
public void testWorkerRegistryClientbasic() {
+ final TaskExecutorContainerConfig containerConfig = TaskExecutorContainerConfig.builder()
+ .taskExecutorThreadPoolSize(10)
+ .containerName("test")
+ .build();
+ final ExclusiveThreadTaskExecutorContainer container =
+ new ExclusiveThreadTaskExecutorContainer(containerConfig);
+ given(physicalTaskExecutorContainerDelegator.getExecutorContainer()).willReturn(container);
+
given(workerConfig.getWorkerAddress()).willReturn(NetUtils.getAddr(1234));
given(workerConfig.getMaxHeartbeatInterval()).willReturn(Duration.ofSeconds(1));
given(workerConfig.getServerLoadProtection()).willReturn(new WorkerServerLoadProtection());
diff --git a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorTest.java
deleted file mode 100644
index 8b527fe5b3..0000000000
--- a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorTest.java
+++ /dev/null
@@ -1,87 +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.worker.runner;
-
-import org.apache.dolphinscheduler.common.constants.Constants;
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
-
-public class DefaultWorkerTaskExecutorTest {
-
- private TaskExecutionContext taskExecutionContext = Mockito.mock(TaskExecutionContext.class);
-
- private WorkerConfig workerConfig = Mockito.mock(WorkerConfig.class);
-
- private String masterAddress = "localhost:5678";
-
- private WorkerMessageSender workerMessageSender = Mockito.mock(WorkerMessageSender.class);
-
- private StorageOperator storageOperator = Mockito.mock(StorageOperator.class);
-
- private WorkerRegistryClient workerRegistryClient = Mockito.mock(WorkerRegistryClient.class);
-
- @Test
- public void testDryRun() {
- TaskExecutionContext taskExecutionContext = TaskExecutionContext.builder()
- .dryRun(Constants.DRY_RUN_FLAG_YES)
- .taskInstanceId(0)
- .workflowDefinitionId(0)
- .firstSubmitTime(System.currentTimeMillis())
- .build();
- WorkerTaskExecutor workerTaskExecutor = new DefaultWorkerTaskExecutor(
- taskExecutionContext,
- workerConfig,
- workerMessageSender,
- storageOperator,
- workerRegistryClient);
-
- Assertions.assertAll(workerTaskExecutor::run);
- Assertions.assertEquals(TaskExecutionStatus.SUCCESS, taskExecutionContext.getCurrentExecutionStatus());
- }
-
- @Test
- public void testErrorboundTestDataSource() {
- TaskExecutionContext taskExecutionContext = TaskExecutionContext.builder()
- .dryRun(Constants.DRY_RUN_FLAG_NO)
- .testFlag(Constants.TEST_FLAG_YES)
- .taskInstanceId(0)
- .workflowDefinitionId(0)
- .firstSubmitTime(System.currentTimeMillis())
- .taskType("SQL")
- .taskParams(
- "{\"localParams\":[],\"resourceList\":[],\"type\":\"POSTGRESQL\",\"datasource\":null,\"sql\":\"select * from t_ds_user\",\"sqlType\":\"0\",\"preStatements\":[],\"postStatements\":[],\"segmentSeparator\":\"\",\"displayRows\":10,\"conditionResult\":\"null\",\"dependence\":\"null\",\"switchResult\":\"null\",\"waitStartTimeout\":null}")
- .build();
- WorkerTaskExecutor workerTaskExecutor = new DefaultWorkerTaskExecutor(
- taskExecutionContext,
- workerConfig,
- workerMessageSender,
- storageOperator,
- workerRegistryClient);
-
- Assertions.assertAll(workerTaskExecutor::run);
- Assertions.assertEquals(TaskExecutionStatus.FAILURE, taskExecutionContext.getCurrentExecutionStatus());
- }
-}
diff --git a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPoolTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPoolTest.java
deleted file mode 100644
index f33bf1faae..0000000000
--- a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorThreadPoolTest.java
+++ /dev/null
@@ -1,177 +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.worker.runner;
-
-import org.apache.dolphinscheduler.common.thread.ThreadUtils;
-import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.worker.config.TaskExecuteThreadsFullPolicy;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-
-import org.apache.commons.lang3.RandomUtils;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
-
-import com.google.common.truth.Truth;
-
-class WorkerTaskExecutorThreadPoolTest {
-
- @BeforeEach
- public void setUp() {
- WorkerTaskExecutorHolder.clear();
- }
-
- @Test
- public void testIsOverload() {
- final int execThreadCount = RandomUtils.nextInt(1, 100);
- final int totalTaskCount = RandomUtils.nextInt(1, 10000);
- final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
- final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
- // submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is CONTINUE
- // after submit execThreadCount task, the thread pool is overload
- for (int i = 1; i <= totalTaskCount; i++) {
- MockWorkerTaskExecutor mockWorkerTaskExecutor =
- new MockWorkerTaskExecutor(() -> ThreadUtils.sleep(10_000L));
- workerTaskExecutorThreadPool.submitWorkerTaskExecutor(mockWorkerTaskExecutor);
- if (i >= execThreadCount) {
- Truth.assertThat(workerTaskExecutorThreadPool.isOverload()).isTrue();
- } else {
- Truth.assertThat(workerTaskExecutorThreadPool.isOverload()).isFalse();
- }
- }
- }
-
- @Test
- public void testSubmitWorkerTaskExecutorWithContinuePolicy() {
- final int execThreadCount = RandomUtils.nextInt(1, 100);
- final int totalTaskCount = RandomUtils.nextInt(1, 10000);
- final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
- final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
- // submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is CONTINUE
- // all task will be submitted success
- for (int i = 1; i <= totalTaskCount; i++) {
- MockWorkerTaskExecutor mockWorkerTaskExecutor =
- new MockWorkerTaskExecutor(() -> ThreadUtils.sleep(10_000L));
- Truth.assertThat(workerTaskExecutorThreadPool.submitWorkerTaskExecutor(mockWorkerTaskExecutor)).isTrue();
- }
- }
-
- @Test
- public void testSubmitWorkerTaskExecutorWithRejectPolicy() {
- final int execThreadCount = RandomUtils.nextInt(1, 100);
- final int totalTaskCount = RandomUtils.nextInt(1, 10000);
- final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.REJECT);
- final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
- // submit totalTaskCount task, the thread pool size is execThreadCount, reject policy is REJECT
- // only the front execThreadCount task will be submitted success
- for (int i = 1; i <= totalTaskCount; i++) {
- MockWorkerTaskExecutor mockWorkerTaskExecutor =
- new MockWorkerTaskExecutor(() -> ThreadUtils.sleep(10_000L));
- boolean submitResult = workerTaskExecutorThreadPool.submitWorkerTaskExecutor(mockWorkerTaskExecutor);
- if (i <= execThreadCount) {
- Assertions.assertTrue(submitResult, "The " + i + " task should submit success");
- } else {
- Assertions.assertFalse(submitResult, "The " + i + " task should submit failed");
- }
- }
- }
-
- @Test
- public void testGetWaitingTaskExecutorSize() {
- final int execThreadCount = RandomUtils.nextInt(1, 100);
- final int totalTaskCount = RandomUtils.nextInt(1, 10000);
- final WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
- final WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
-
- Truth.assertThat(workerTaskExecutorThreadPool.getWaitingTaskExecutorSize()).isEqualTo(0);
- for (int i = 1; i <= totalTaskCount; i++) {
- MockWorkerTaskExecutor mockWorkerTaskExecutor =
- new MockWorkerTaskExecutor(() -> ThreadUtils.sleep(10_000L));
- workerTaskExecutorThreadPool.submitWorkerTaskExecutor(mockWorkerTaskExecutor);
- if (i <= execThreadCount) {
- Truth.assertThat(workerTaskExecutorThreadPool.getWaitingTaskExecutorSize()).isEqualTo(0);
- } else {
- Truth.assertThat(workerTaskExecutorThreadPool.getWaitingTaskExecutorSize())
- .isEqualTo(i - execThreadCount);
- }
- }
- }
-
- @Test
- public void testGetRunningTaskExecutorSize() {
- final int execThreadCount = RandomUtils.nextInt(1, 100);
- final int totalTaskCount = RandomUtils.nextInt(1, 10000);
- WorkerConfig workerConfig = createWorkerConfig(execThreadCount, TaskExecuteThreadsFullPolicy.CONTINUE);
- WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- new WorkerTaskExecutorThreadPool(workerConfig, Mockito.mock(WorkerMessageSender.class));
-
- Truth.assertThat(workerTaskExecutorThreadPool.getRunningTaskExecutorSize()).isEqualTo(0);
- for (int i = 1; i <= totalTaskCount; i++) {
- MockWorkerTaskExecutor mockWorkerTaskExecutor =
- new MockWorkerTaskExecutor(() -> ThreadUtils.sleep(10_000L));
- workerTaskExecutorThreadPool.submitWorkerTaskExecutor(mockWorkerTaskExecutor);
- if (i <= execThreadCount) {
- Truth.assertThat(workerTaskExecutorThreadPool.getRunningTaskExecutorSize()).isEqualTo(i);
- } else {
- Truth.assertThat(workerTaskExecutorThreadPool.getRunningTaskExecutorSize()).isEqualTo(execThreadCount);
- }
- }
- }
-
- static class MockWorkerTaskExecutor extends WorkerTaskExecutor {
-
- private final Runnable runnable;
-
- protected MockWorkerTaskExecutor(Runnable runnable) {
- super(TaskExecutionContext.builder().taskInstanceId((int) System.nanoTime()).build(),
- new WorkerConfig(),
- new WorkerMessageSender(),
- null,
- new WorkerRegistryClient());
- this.runnable = runnable;
- }
-
- @Override
- public void run() {
- executeTask(new TaskCallbackImpl(null, null));
- }
-
- @Override
- protected void executeTask(TaskCallBack taskCallBack) {
- runnable.run();
- }
- }
-
- private WorkerConfig createWorkerConfig(int execThreads,
- TaskExecuteThreadsFullPolicy taskExecuteThreadsFullPolicy) {
- WorkerConfig workerConfig = new WorkerConfig();
- workerConfig.setExecThreads(execThreads);
- workerConfig.setTaskExecuteThreadsFullPolicy(taskExecuteThreadsFullPolicy);
- return workerConfig;
- }
-
-}
diff --git a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionTest.java
deleted file mode 100644
index a8edf4d5a6..0000000000
--- a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/operator/TaskInstanceOperationFunctionTest.java
+++ /dev/null
@@ -1,275 +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.worker.runner.operator;
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-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;
-import org.apache.dolphinscheduler.plugin.storage.api.StorageOperator;
-import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
-import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
-import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner;
-import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
-import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutor;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorFactoryBuilder;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorHolder;
-import org.apache.dolphinscheduler.server.worker.runner.WorkerTaskExecutorThreadPool;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.mockito.MockedStatic;
-import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TaskInstanceOperationFunctionTest {
-
- private static final Logger log = LoggerFactory.getLogger(TaskInstanceOperationFunctionTest.class);
- private MessageRetryRunner messageRetryRunner = Mockito.mock(MessageRetryRunner.class);
-
- private WorkerConfig workerConfig = Mockito.mock(WorkerConfig.class);
-
- private TaskExecutionContext taskExecutionContext = Mockito.mock(TaskExecutionContext.class);
-
- private WorkerTaskExecutorThreadPool workerTaskExecutorThreadPool =
- Mockito.mock(WorkerTaskExecutorThreadPool.class);
-
- private WorkerTaskExecutor workerTaskExecutor = Mockito.mock(WorkerTaskExecutor.class);
-
- private AbstractTask task = Mockito.mock(AbstractTask.class);
-
- private WorkerMessageSender workerMessageSender = Mockito.mock(WorkerMessageSender.class);
-
- private WorkerTaskExecutorThreadPool workerManager = Mockito.mock(WorkerTaskExecutorThreadPool.class);
-
- private StorageOperator storageOperator = Mockito.mock(StorageOperator.class);
-
- private WorkerRegistryClient workerRegistryClient = Mockito.mock(WorkerRegistryClient.class);
-
- @Test
- public void testTaskInstanceOperationFunctionManager() {
- TaskInstanceKillOperationFunction taskInstanceKillOperationFunction = new TaskInstanceKillOperationFunction(
- workerTaskExecutorThreadPool,
- messageRetryRunner);
-
- TaskInstancePauseOperationFunction taskInstancePauseOperationFunction =
- new TaskInstancePauseOperationFunction();
-
- TakeOverTaskOperationFunction takeOverTaskOperationFunction =
- new TakeOverTaskOperationFunction(
- messageRetryRunner);
-
- WorkerTaskExecutorFactoryBuilder workerTaskExecutorFactoryBuilder = new WorkerTaskExecutorFactoryBuilder(
- workerConfig,
- workerMessageSender,
- workerManager,
- storageOperator,
- workerRegistryClient);
-
- TaskInstanceDispatchOperationFunction taskInstanceDispatchOperationFunction =
- new TaskInstanceDispatchOperationFunction(
- workerConfig,
- workerTaskExecutorFactoryBuilder,
- workerTaskExecutorThreadPool);
-
- TaskInstanceOperationFunctionManager taskInstanceOperationFunctionManager =
- new TaskInstanceOperationFunctionManager(
- taskInstanceKillOperationFunction,
- takeOverTaskOperationFunction,
- taskInstanceDispatchOperationFunction,
- taskInstancePauseOperationFunction);
-
- Assertions.assertEquals(taskInstanceKillOperationFunction,
- taskInstanceOperationFunctionManager.getTaskInstanceKillOperationFunction());
- Assertions.assertEquals(taskInstancePauseOperationFunction,
- taskInstanceOperationFunctionManager.getTaskInstancePauseOperationFunction());
- Assertions.assertEquals(takeOverTaskOperationFunction,
- taskInstanceOperationFunctionManager.getUpdateWorkflowHostOperationFunction());
- Assertions.assertEquals(taskInstanceDispatchOperationFunction,
- taskInstanceOperationFunctionManager.getTaskInstanceDispatchOperationFunction());
- }
-
- @Test
- public void testUpdateWorkflowHostOperationFunction() {
- TakeOverTaskOperationFunction takeOverTaskOperationFunction =
- new TakeOverTaskOperationFunction(
- messageRetryRunner);
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceIdMDC(any(Integer.class)))
- .then(invocationOnMock -> null);
- TakeOverTaskRequest request = new TakeOverTaskRequest();
- request.setTaskInstanceId(1);
- request.setWorkflowHost("host");
- TakeOverTaskResponse taskInstanceDispatchResponse = takeOverTaskOperationFunction.operate(
- request);
- Assertions.assertEquals(taskInstanceDispatchResponse.isSuccess(), false);
- }
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceIdMDC(any(Integer.class)))
- .then(invocationOnMock -> null);
-
- try (
- MockedStatic workerTaskExecutorHolderMockedStatic =
- Mockito.mockStatic(WorkerTaskExecutorHolder.class)) {
- given(workerTaskExecutor.getTaskExecutionContext()).willReturn(taskExecutionContext);
- workerTaskExecutorHolderMockedStatic
- .when(() -> WorkerTaskExecutorHolder.get(any(Integer.class)))
- .thenReturn(workerTaskExecutor);
- int taskInstanceId = 111;
- TakeOverTaskRequest request = new TakeOverTaskRequest();
- request.setTaskInstanceId(taskInstanceId);
- request.setWorkflowHost("host");
-
- TakeOverTaskResponse taskInstanceDispatchResponse = takeOverTaskOperationFunction.operate(
- request);
- Assertions.assertEquals(taskInstanceDispatchResponse.isSuccess(), true);
- }
- }
- }
-
- @Test
- public void testTaskInstancePauseOperationFunction() {
- TaskInstancePauseOperationFunction taskInstancePauseOperationFunction =
- new TaskInstancePauseOperationFunction();
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceIdMDC(any(Integer.class)))
- .then(invocationOnMock -> null);
- TaskInstancePauseRequest request = new TaskInstancePauseRequest();
- request.setTaskInstanceId(1);
- TaskInstancePauseResponse taskInstanceDispatchResponse = taskInstancePauseOperationFunction.operate(
- request);
- Assertions.assertEquals(taskInstanceDispatchResponse.isSuccess(), true);
- }
- }
-
- @Test
- public void testTaskInstanceDispatchOperationFunction() {
- WorkerTaskExecutorFactoryBuilder workerTaskExecutorFactoryBuilder = new WorkerTaskExecutorFactoryBuilder(
- workerConfig,
- workerMessageSender,
- workerManager,
- storageOperator,
- workerRegistryClient);
-
- TaskInstanceDispatchOperationFunction taskInstanceDispatchOperationFunction =
- new TaskInstanceDispatchOperationFunction(
- workerConfig,
- workerTaskExecutorFactoryBuilder,
- workerTaskExecutorThreadPool);
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- logUtilsMockedStatic
- .when(() -> LogUtils
- .getTaskInstanceLogFullPath(any(TaskExecutionContext.class)))
- .thenReturn("test");
- TaskInstanceDispatchResponse taskInstanceDispatchResponse = taskInstanceDispatchOperationFunction.operate(
- new TaskInstanceDispatchRequest(taskExecutionContext));
- Assertions.assertEquals(taskInstanceDispatchResponse.isDispatchSuccess(), false);
- logUtilsMockedStatic.verify(times(1), () -> LogUtils.removeWorkflowAndTaskInstanceIdMDC());
-
- given(workerTaskExecutorThreadPool.submitWorkerTaskExecutor(any())).willReturn(true);
- taskInstanceDispatchResponse = taskInstanceDispatchOperationFunction.operate(
- new TaskInstanceDispatchRequest(taskExecutionContext));
- Assertions.assertEquals(taskInstanceDispatchResponse.isDispatchSuccess(), true);
- logUtilsMockedStatic.verify(times(2), () -> LogUtils.removeWorkflowAndTaskInstanceIdMDC());
- }
- }
-
- @Test
- public void testTaskInstanceKillOperationFunction() {
- TaskInstanceKillOperationFunction taskInstanceKillOperationFunction = new TaskInstanceKillOperationFunction(
- workerManager,
- messageRetryRunner);
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- int taskInstanceId = 111;
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceLogFullPathMDC(any(String.class)))
- .then(invocationOnMock -> null);
- TaskInstanceKillResponse response = taskInstanceKillOperationFunction.operate(
- new TaskInstanceKillRequest(taskInstanceId));
- Assertions.assertEquals("Cannot find WorkerTaskExecutor", response.getMessage());
- }
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- int processId = 12;
- int taskInstanceId = 111;
- Mockito.reset(taskExecutionContext);
- given(taskExecutionContext.getProcessId()).willReturn(processId);
- given(taskExecutionContext.getLogPath()).willReturn("logpath");
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceLogFullPathMDC(any(String.class)))
- .then(invocationOnMock -> null);
- taskInstanceKillOperationFunction.operate(
- new TaskInstanceKillRequest(taskInstanceId));
- logUtilsMockedStatic.verify(times(1), () -> LogUtils.removeTaskInstanceIdMDC());
- logUtilsMockedStatic.verify(times(1), () -> LogUtils.removeTaskInstanceLogFullPathMDC());
- }
-
- try (MockedStatic logUtilsMockedStatic = Mockito.mockStatic(LogUtils.class)) {
- try (
- MockedStatic workerTaskExecutorHolderMockedStatic =
- Mockito.mockStatic(WorkerTaskExecutorHolder.class)) {
- given(workerTaskExecutor.getTaskExecutionContext()).willReturn(taskExecutionContext);
- workerTaskExecutorHolderMockedStatic
- .when(() -> WorkerTaskExecutorHolder.get(any(Integer.class)))
- .thenReturn(workerTaskExecutor);
- int processId = 12;
- int taskInstanceId = 111;
- Mockito.reset(taskExecutionContext);
- given(taskExecutionContext.getProcessId()).willReturn(processId);
- given(taskExecutionContext.getLogPath()).willReturn("logpath");
- logUtilsMockedStatic
- .when(() -> LogUtils
- .setTaskInstanceLogFullPathMDC(any(String.class)))
- .then(invocationOnMock -> null);
- when(workerTaskExecutor.getTask()).thenReturn(task);
- // given(workerManager.getTaskExecuteThread(taskInstanceId)).willReturn(workerTaskExecutor);
- taskInstanceKillOperationFunction.operate(
- new TaskInstanceKillRequest(taskInstanceId));
- verify(task, times(1)).cancel();
- }
-
- }
- }
-}
diff --git a/pom.xml b/pom.xml
index 8ad04e8bce..fcbc20c820 100755
--- a/pom.xml
+++ b/pom.xml
@@ -58,6 +58,7 @@
dolphinscheduler-dao-plugin
dolphinscheduler-authentication
dolphinscheduler-eventbus
+ dolphinscheduler-task-executor
@@ -273,6 +274,11 @@
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-extract-common
+ ${project.version}
+
org.apache.dolphinscheduler
dolphinscheduler-extract-master
@@ -325,6 +331,18 @@
${project.version}
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-eventbus
+ ${project.version}
+
+
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+ ${project.version}
+
+
org.junit
junit-bom