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 a3137d3f7ca2a..a5f28e56f83ac 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
@@ -47,14 +47,14 @@
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;
@@ -306,11 +306,11 @@ public Result stopTask(User loginUser, long projectCode, Integer taskInstanceId)
}
// 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-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 932da89d6b978..1c3cfc86b7439 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 @@
@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 static ScheduledExecutorService newSingleDaemonScheduledExecutorService(S
* 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 bc57aef9e9667..05ea7006b82ed 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,26 +20,47 @@
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
+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.
*/
+@SuperBuilder
public abstract class AbstractDelayEvent implements IEvent, Delayed {
protected long delayTime;
protected long triggerTimeInMillis;
+ @Deprecated
+ protected RetryPolicy retryPolicy;
+
public AbstractDelayEvent() {
this(0);
}
public AbstractDelayEvent(long delayTime) {
+ this(delayTime, null);
+ }
+
+ public AbstractDelayEvent(final long delayTime, final RetryPolicy retryPolicy) {
if (delayTime == 0) {
this.triggerTimeInMillis = System.currentTimeMillis();
} else {
this.triggerTimeInMillis = System.currentTimeMillis() + delayTime;
}
+ this.retryPolicy = retryPolicy;
+ }
+
+ @Override
+ public boolean isRetryable() {
+ return retryPolicy != null;
+ }
+
+ @Override
+ public RetryPolicy getRetryPolicy() {
+ return retryPolicy;
}
@Override
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 4d3c259e4596b..81809c546c842 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 Optional poll() {
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/IEvent.java b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEvent.java
index f28f8f4fa48d5..5106143b635a1 100644
--- a/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEvent.java
+++ b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/IEvent.java
@@ -22,4 +22,37 @@
*/
public interface IEvent {
+ /**
+ * Whether the event can retry, if true the event will be stored in the event bus until the retry policy break.
+ */
+ boolean isRetryable();
+
+ /**
+ * Get the retry policy.
+ * Use the control retry times and interval.
+ */
+ RetryPolicy getRetryPolicy();
+
+ interface RetryPolicy {
+
+ /**
+ * Whether the retry policy has been broken.
+ */
+ boolean isRetryPolicyBroken();
+
+ /**
+ * Increase the retry times.
+ */
+ void increaseRetryTimes();
+
+ void setNextRetryTimeStampWithFixedStep();
+
+ void setNextRetryTimeStampWithExponentialStep();
+
+ /**
+ * Get the next time which the event can retry.
+ */
+ long getNextRetryTimeStamp();
+
+ }
}
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 8a8c458e6abdd..b2f6577a41457 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-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/LinearTimeRetryPolicy.java b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/LinearTimeRetryPolicy.java
new file mode 100644
index 0000000000000..e5b497e14f017
--- /dev/null
+++ b/dolphinscheduler-eventbus/src/main/java/org/apache/dolphinscheduler/eventbus/LinearTimeRetryPolicy.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.eventbus;
+
+import java.time.Duration;
+
+public class LinearTimeRetryPolicy implements IEvent.RetryPolicy {
+
+ private int currentRetryTimes = 0;
+
+ private final long firstRetryTimeStamp = System.currentTimeMillis();
+
+ private long nextRetryTimeStamp = firstRetryTimeStamp;
+
+ private static final int maxRetryTimes = 180;
+
+ private static final long maxRetryDuration = Duration.ofDays(1).toMillis();
+
+ private static final long baseRetryInterval = Duration.ofSeconds(10).toMillis();
+
+ @Override
+ public boolean isRetryPolicyBroken() {
+ if (currentRetryTimes > maxRetryTimes) {
+ return true;
+ }
+ if (nextRetryTimeStamp > firstRetryTimeStamp + maxRetryDuration) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public void increaseRetryTimes() {
+ currentRetryTimes++;
+ setNextRetryTimeStampWithExponentialStep();
+ }
+
+ @Override
+ public void setNextRetryTimeStampWithFixedStep() {
+ nextRetryTimeStamp += baseRetryInterval;
+ }
+
+ @Override
+ public void setNextRetryTimeStampWithExponentialStep() {
+ nextRetryTimeStamp += baseRetryInterval * currentRetryTimes;
+ }
+
+ @Override
+ public long getNextRetryTimeStamp() {
+ return nextRetryTimeStamp;
+ }
+
+ @Override
+ public String toString() {
+ return "LinearTimeRetryPolicy{" +
+ "firstRetryTimeStamp=" + firstRetryTimeStamp +
+ ", currentRetryTimes=" + currentRetryTimes +
+ '}';
+ }
+}
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 56e19fe8b2083..3450192f62689 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 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.json.JsonMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
-/**
- * json serialize or deserialize
- */
@Slf4j
public class JsonSerializer {
@@ -60,13 +57,6 @@ private 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 static byte[] serialize(T obj) {
}
}
- /**
- * 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/JdkDynamicServerHandler.java b/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/JdkDynamicServerHandler.java
index 4f9a7034c8c0c..2bf1916898f1e 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/JdkDynamicServerHandler.java
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/JdkDynamicServerHandler.java
@@ -102,7 +102,8 @@ private void processReceived(final Channel channel, final Transporter transporte
} else {
args = new Object[standardRpcRequest.getArgs().length];
for (int i = 0; i < standardRpcRequest.getArgs().length; i++) {
- args[i] = JsonSerializer.deserialize(standardRpcRequest.getArgs()[i],
+ args[i] = JsonSerializer.deserialize(
+ standardRpcRequest.getArgs()[i],
standardRpcRequest.getArgsTypes()[i]);
}
}
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 c602f15bb255e..4484e519421b4 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 @@
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 955bcd0a12178..729238a490ba6 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-master/pom.xml
@@ -30,21 +30,31 @@
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
+ ${project.version}
+
+
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 0000000000000..dfbbb6b37f1ec
--- /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.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 f85200b7a2c68..0000000000000
--- 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 7ebd1310bd929..0000000000000
--- 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 0000000000000..441c41ec0d27b
--- /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.event.TaskExecutorDispatchedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPausedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorRuntimeContextChangedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorStartedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.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 0000000000000..122fcd20bed12
--- /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.ITaskExecutorEventRemoteReporterClient;
+import org.apache.dolphinscheduler.task.executor.event.IReportableTaskExecutorLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorDispatchedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPausedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorRuntimeContextChangedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorStartedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.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 a1df0829a588f..8d9f4f7a90268 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 7f2b16a530a76..0000000000000
--- 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/TaskExecutionPausedEvent.java b/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionPausedEvent.java
deleted file mode 100644
index 6186d0b76228a..0000000000000
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionPausedEvent.java
+++ /dev/null
@@ -1,49 +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 TaskExecutionPausedEvent implements ITaskExecutionEvent {
-
- private int taskInstanceId;
-
- private int workflowInstanceId;
-
- private String taskInstanceHost;
-
- private String workflowInstanceHost;
-
- private long endTime;
-
- private long eventCreateTime;
-
- private long eventSendTime;
-
- @Override
- public TaskInstanceExecutionEventType getEventType() {
- return TaskInstanceExecutionEventType.PAUSED;
- }
-}
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 a7dd1c8909af9..0000000000000
--- 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 0a42b73a1da79..b8cbca09caee3 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
+++ b/dolphinscheduler-extract/dolphinscheduler-extract-worker/pom.xml
@@ -30,11 +30,13 @@
dolphinscheduler-extract-worker
+
org.apache.dolphinscheduler
dolphinscheduler-extract-common
${project.version}
+
org.apache.dolphinscheduler
dolphinscheduler-extract-base
@@ -45,6 +47,12 @@
dolphinscheduler-task-api
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+ ${project.version}
+
+
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 0000000000000..5495b5a786a12
--- /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.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 fda644110ffd2..0000000000000
--- 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 43d6a39b010b6..0000000000000
--- 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 3c02bbe461722..0000000000000
--- 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 38b6441c1f6aa..0000000000000
--- 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 fee22e9431d44..0000000000000
--- 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 bcaba1d4fd943..0000000000000
--- 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 8845ed9081748..0000000000000
--- 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 088058284ac38..0000000000000
--- 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 90ae49ab0eb02..0000000000000
--- 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 aaeb4eb6908c3..0000000000000
--- 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 2999a299bd4e0..0000000000000
--- 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 3d649fa8d8099..c86e70268fc8c 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.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 82093e1cf9085..9d70b002f0220 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.Data;
import lombok.NoArgsConstructor;
+@Deprecated
@Data
@NoArgsConstructor
@AllArgsConstructor
diff --git a/dolphinscheduler-master/pom.xml b/dolphinscheduler-master/pom.xml
index 1cf68d216fc4c..7cc7fa5962a55 100644
--- a/dolphinscheduler-master/pom.xml
+++ b/dolphinscheduler-master/pom.xml
@@ -89,6 +89,13 @@
dolphinscheduler-eventbus
${project.version}
+
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-task-executor
+ ${project.version}
+
+
org.codehaus.janino
janino
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseRequest.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.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-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java
index 0ac51e0212c1d..47d73755c25c7 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseRequest.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/LogicTaskConfig.java
@@ -15,17 +15,13 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.extract.master.transportor;
+package org.apache.dolphinscheduler.server.master.config;
-import lombok.AllArgsConstructor;
import lombok.Data;
-import lombok.NoArgsConstructor;
@Data
-@NoArgsConstructor
-@AllArgsConstructor
-public class LogicTaskPauseRequest {
-
- private int taskInstanceId;
+public class LogicTaskConfig {
+ private int taskExecutorThreadCount = Runtime.getRuntime().availableProcessors() * 2 + 1;
+ private int taskExecutorEventBusFireThreadCount = 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 9ff59e0cdb94c..02c739953d594 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
@@ -49,10 +49,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.
*/
@@ -118,6 +116,7 @@ private void printConfig() {
"\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 registry-disconnect-strategy -> " + registryDisconnectStrategy +
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 905a5d2fa8adc..6e76b30c38d87 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
@@ -126,7 +126,7 @@ public void run() {
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 @@ private void amendTaskGroupQueueStatus() {
}
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 @@ private void dealWithForceStartTaskGroupQueue() {
}
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/WorkflowEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/WorkflowEngine.java
index 76ca89dd577cb..b86ef48084402 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.LogicTaskEngine;
+import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueueLooper;
import lombok.extern.slf4j.Slf4j;
@@ -36,21 +37,26 @@ public class WorkflowEngine implements AutoCloseable {
private WorkflowEventBusCoordinator workflowEventBusCoordinator;
@Autowired
- private MasterTaskExecutorBootstrap masterTaskExecutorBootstrap;
+ private CommandEngine commandEngine;
@Autowired
- private CommandEngine commandEngine;
+ private GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper;
+
+ @Autowired
+ private LogicTaskEngine logicTaskEngine;
public void start() {
taskGroupCoordinator.start();
- masterTaskExecutorBootstrap.start();
-
workflowEventBusCoordinator.start();
commandEngine.start();
+ globalTaskDispatchWaitingQueueLooper.start();
+
+ logicTaskEngine.start();
+
log.info("WorkflowEngine started");
}
@@ -59,8 +65,10 @@ public void close() throws Exception {
try (
final CommandEngine commandEngine1 = commandEngine;
final WorkflowEventBusCoordinator workflowEventBusCoordinator1 = workflowEventBusCoordinator;
- final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap;
- final TaskGroupCoordinator taskGroupCoordinator1 = taskGroupCoordinator) {
+ final GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper1 =
+ globalTaskDispatchWaitingQueueLooper;
+ final TaskGroupCoordinator taskGroupCoordinator1 = taskGroupCoordinator;
+ final LogicTaskEngine logicTaskEngine1 = logicTaskEngine) {
// closed the resource
}
}
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 cfd13b4e45e1c..f76a0dfa470b9 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 List fetchCommands() {
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/TaskExecutorEventAckException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskExecutorEventAckException.java
new file mode 100644
index 0000000000000..35f5dcb27d12d
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskExecutorEventAckException.java
@@ -0,0 +1,29 @@
+/*
+ * 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 TaskExecutorEventAckException extends RuntimeException {
+
+ public TaskExecutorEventAckException(String message) {
+ super(message);
+ }
+
+ public TaskExecutorEventAckException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskTakeOverException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskTakeOverException.java
new file mode 100644
index 0000000000000..61c1ba6ad034a
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/exceptions/TaskTakeOverException.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 TaskTakeOverException extends RuntimeException {
+
+ public TaskTakeOverException(String message) {
+ super(message);
+ }
+
+ public TaskTakeOverException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngine.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngine.java
new file mode 100644
index 0000000000000..58cc6a1eb1821
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngine.java
@@ -0,0 +1,79 @@
+/*
+ * 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.ITaskExecutorLifecycleEventReporter;
+import org.apache.dolphinscheduler.task.executor.TaskEngine;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.stereotype.Component;
+
+@Slf4j
+@Component
+public class LogicTaskEngine implements AutoCloseable {
+
+ private final TaskEngine taskEngine;
+
+ private final LogicTaskExecutorFactory logicTaskExecutorFactory;
+
+ private final LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter;
+
+ public LogicTaskEngine(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("LogicTaskEngine 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 (
+ TaskEngine taskEngine1 = taskEngine;
+ LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter1 =
+ logicTaskExecutorEventReporter) {
+ log.info("LogicTaskEngine closed");
+ }
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperationFunctionManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java
similarity index 51%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperationFunctionManager.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java
index de4095831e9b5..c44bfbbdd3dfa 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperationFunctionManager.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskEngineFactory.java
@@ -15,33 +15,33 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.rpc;
+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.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
-public class LogicTaskInstanceOperationFunctionManager {
+public class LogicTaskEngineFactory {
@Autowired
- private LogicITaskInstanceDispatchOperationFunction logicITaskInstanceDispatchOperationFunction;
+ private LogicTaskExecutorRepository logicTaskExecutorRepository;
@Autowired
- private LogicITaskInstanceKillOperationFunction logicITaskInstanceKillOperationFunction;
+ private LogicTaskExecutorContainerDelegator logicTaskExecutorContainerDelegator;
@Autowired
- private LogicITaskInstancePauseOperationFunction logicITaskInstancePauseOperationFunction;
-
- public LogicITaskInstanceDispatchOperationFunction getLogicTaskInstanceDispatchOperationFunction() {
- return logicITaskInstanceDispatchOperationFunction;
- }
-
- public LogicITaskInstanceKillOperationFunction getLogicTaskInstanceKillOperationFunction() {
- return logicITaskInstanceKillOperationFunction;
- }
-
- public LogicITaskInstancePauseOperationFunction getLogicTaskInstancePauseOperationFunction() {
- return logicITaskInstancePauseOperationFunction;
+ private LogicTaskExecutorEventBusCoordinator logicTaskExecutorEventBusCoordinator;
+
+ public TaskEngine createTaskEngine() {
+ final TaskEngineBuilder taskEngineBuilder = TaskEngineBuilder.builder()
+ .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 0000000000000..fca122c91d2fd
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.engine.executor;
+
+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.log.TaskInstanceLogHeader;
+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.TaskExecutorEventBus;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorState;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorStartedLifecycleEvent;
+
+import lombok.Getter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class LogicTaskExecutor extends AbstractTaskExecutor {
+
+ @Getter
+ private final TaskExecutionContext taskExecutionContext;
+
+ @Getter
+ private final TaskExecutorEventBus taskExecutorEventBus;
+
+ private ILogicTask logicTask;
+
+ private final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
+
+ public LogicTaskExecutor(final LogicTaskExecutorBuilder logicTaskExecutorBuilder) {
+ super();
+ this.taskExecutionContext = logicTaskExecutorBuilder.getTaskExecutionContext();
+ this.logicTaskPluginFactoryBuilder = logicTaskExecutorBuilder.getLogicTaskPluginFactoryBuilder();
+ this.taskExecutorEventBus = logicTaskExecutorBuilder.getTaskExecutorEventBus();
+ }
+
+ @SneakyThrows
+ @Override
+ public void start() {
+
+ TaskInstanceLogHeader.printInitializeTaskContextHeader();
+ initializeTaskContext();
+
+ publishTaskRunningEvent();
+
+ TaskInstanceLogHeader.printLoadTaskInstancePluginHeader();
+ initializeLogicTaskPlugin();
+
+ TaskInstanceLogHeader.printExecuteTaskHeader();
+ logicTask.start();
+
+ }
+
+ @Override
+ protected TaskExecutionStatus doTrackTaskPluginStatus() {
+ return logicTask.getTaskExecutionState();
+ }
+
+ @SneakyThrows
+ @Override
+ public void pause() {
+ // todo: 判断当前任务是否开始执行
+ if (logicTask != null) {
+ logicTask.pause();
+ }
+ }
+
+ @SneakyThrows
+ @Override
+ public void kill() {
+ // todo: 判断当前任务是否开始执行
+ if (logicTask != null) {
+ logicTask.kill();
+ }
+ }
+
+ private void initializeTaskContext() {
+ log.info("Begin to initialize taskContext");
+ taskExecutionContext.setStartTime(System.currentTimeMillis());
+ log.info("End initialize task {}", JSONUtils.toPrettyJsonString(taskExecutionContext));
+ }
+
+ private void publishTaskRunningEvent() {
+ transitTaskExecutorState(TaskExecutorState.RUNNING);
+ taskExecutorEventBus.publish(TaskExecutorStartedLifecycleEvent.of(this));
+ }
+
+ @SneakyThrows
+ private void initializeLogicTaskPlugin() {
+ logicTask = logicTaskPluginFactoryBuilder
+ .createILogicTaskPluginFactory(taskExecutionContext.getTaskType())
+ .createLogicTask(this);
+ log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType());
+ }
+
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchRequest.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
similarity index 67%
rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchRequest.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
index ca1064515237a..e3acc3ba02ddd 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceDispatchRequest.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorBuilder.java
@@ -15,20 +15,26 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.extract.worker.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.TaskExecutorEventBus;
import lombok.AllArgsConstructor;
+import lombok.Builder;
import lombok.Data;
-import lombok.NoArgsConstructor;
@Data
-@NoArgsConstructor
+@Builder
@AllArgsConstructor
-public class TaskInstanceDispatchRequest {
-
- private static final long serialVersionUID = -1L;
+public class LogicTaskExecutorBuilder {
private TaskExecutionContext taskExecutionContext;
+
+ @Builder.Default
+ private TaskExecutorEventBus taskExecutorEventBus = new TaskExecutorEventBus();
+
+ private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
+
}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerDelegator.java
new file mode 100644
index 0000000000000..7dbf4a503e208
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorContainerDelegator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.LogicTaskConfig;
+import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer;
+import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerDelegator;
+import org.apache.dolphinscheduler.task.executor.container.SharedThreadTaskExecutorContainer;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerBuilder;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerConfig;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class LogicTaskExecutorContainerDelegator implements ITaskExecutorContainerDelegator {
+
+ private final ITaskExecutorContainer taskExecutorContainer;
+
+ public LogicTaskExecutorContainerDelegator(final MasterConfig masterConfig) {
+ final LogicTaskConfig logicTaskConfig = masterConfig.getLogicTaskConfig();
+ final TaskExecutorContainerBuilder taskExecutorContainerBuilder = TaskExecutorContainerBuilder.builder()
+ .taskExecutorContainerName("SharedThreadTaskExecutorContainer")
+ .taskExecutorContainerConfig(
+ new TaskExecutorContainerConfig(logicTaskConfig.getTaskExecutorThreadCount()))
+ .build();
+ this.taskExecutorContainer = new SharedThreadTaskExecutorContainer(taskExecutorContainerBuilder);
+ }
+
+ @Override
+ public ITaskExecutorContainer getExecutorContainer(final String taskType) {
+ return taskExecutorContainer;
+ }
+}
diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
similarity index 54%
rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
index 68ebfba29896c..96fe33440efd7 100644
--- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchResponse.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorEventBusCoordinator.java
@@ -15,28 +15,18 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.extract.master.transportor;
+package org.apache.dolphinscheduler.server.master.engine.executor;
-import lombok.AllArgsConstructor;
-import lombok.Data;
-import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorEventBusCoordinator;
-@Data
-@NoArgsConstructor
-@AllArgsConstructor
-public class LogicTaskDispatchResponse {
+import org.springframework.stereotype.Component;
- private Integer taskInstanceId;
+@Component
+public class LogicTaskExecutorEventBusCoordinator 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 LogicTaskExecutorEventBusCoordinator(final LogicTaskExecutorRepository logicTaskExecutorRepository,
+ final LogicTaskExecutorLifecycleEventListener logicTaskExecutorLifecycleEventListener) {
+ super(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 0000000000000..4cb7551ba9c3c
--- /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 0000000000000..355fdf76bca09
--- /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/runner/dispatcher/TaskDispatchFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java
similarity index 50%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java
index 9943ee1b5bfe6..8f1a77e9de64c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorLifecycleEventListener.java
@@ -15,35 +15,22 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.runner.dispatcher;
+package org.apache.dolphinscheduler.server.master.engine.executor;
-import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
+import org.apache.dolphinscheduler.task.executor.listener.TaskExecutorLifecycleEventListener;
-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;
-
- @Autowired
- private WorkerTaskDispatcher workerTaskDispatcher;
-
- public TaskDispatcher getTaskDispatcher(String taskType) {
- if (TaskTypeUtils.isLogicTask(taskType)) {
- return masterTaskDispatcher;
- }
- return workerTaskDispatcher;
+public class LogicTaskExecutorLifecycleEventListener extends TaskExecutorLifecycleEventListener {
+
+ public LogicTaskExecutorLifecycleEventListener(
+ final LogicTaskExecutorContainerDelegator logicTaskExecutorContainerDelegator,
+ final LogicTaskExecutorRepository logicTaskExecutorRepository,
+ final LogicTaskExecutorLifecycleEventReporter logicTaskExecutorEventReporter) {
+ super(
+ logicTaskExecutorContainerDelegator,
+ logicTaskExecutorRepository,
+ logicTaskExecutorEventReporter);
}
-
- public TaskDispatcher getTaskDispatcher(TaskInstance taskInstance) {
- return getTaskDispatcher(taskInstance.getTaskType());
- }
-
}
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 0000000000000..fa0d3f08308af
--- /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.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 0000000000000..c6669305431ed
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/LogicTaskExecutorRepository.java
@@ -0,0 +1,61 @@
+/*
+ * 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 static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
+import org.apache.dolphinscheduler.task.executor.ITaskExecutorRepository;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class LogicTaskExecutorRepository implements ITaskExecutorRepository {
+
+ private final Map taskExecutorMap = new ConcurrentHashMap<>();
+
+ @Override
+ public void put(final ITaskExecutor taskExecutor) {
+ checkNotNull(taskExecutor);
+ taskExecutorMap.put(taskExecutor.getId(), taskExecutor);
+ }
+
+ @Override
+ public ITaskExecutor get(final Integer taskExecutorId) {
+ return taskExecutorMap.get(taskExecutorId);
+ }
+
+ @Override
+ public Collection getAll() {
+ return taskExecutorMap.values();
+ }
+
+ @Override
+ public boolean contains(final Integer taskExecutorId) {
+ return taskExecutorMap.containsKey(taskExecutorId);
+ }
+
+ @Override
+ public void remove(final Integer taskExecutorId) {
+ taskExecutorMap.remove(taskExecutorId);
+ }
+}
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 0000000000000..8075b00fde56d
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/AbstractLogicTask.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.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));
+ taskExecutionStatus = TaskExecutionStatus.RUNNING_EXECUTION;
+ }
+
+ @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 806792f45ae29..5052b0b73c0fb 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 80%
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 08b2285184002..e35fc9accf1e3 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,14 @@
* 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.server.master.exception.LogicTaskInitializeException;
+import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
public interface ILogicTaskPluginFactory {
- T createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException;
+ T createLogicTask(final ITaskExecutor taskExecutor) throws LogicTaskInitializeException;
String getTaskType();
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/ITaskInstanceOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java
similarity index 83%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/ITaskInstanceOperationFunction.java
rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/ITaskParameterDeserializer.java
index 640ee6be12786..31d4aa5bf040d 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/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.master.rpc;
+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 96%
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 7d6cc7260cdb8..e6580fa627138 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,7 +15,7 @@
* 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.server.master.exception.LogicTaskFactoryNotFoundException;
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 6088ff53ea48d..dcfca074e4404 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 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 @@ private DependResult getDependResultForItem(DependentItem item, Map 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 dc797c6a68b16..7c4ee34895c41 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 70ac2ba10bb06..3e4d7314f0c79 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.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 DependentLogicTask(TaskExecutionContext taskExecutionContext,
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 31c4f67bf633b..6c4f35b1fb493 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.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 DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionCont
@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 bf8752a98db99..f2cae5828e404 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.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.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 DependentAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionConte
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 DependentAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionConte
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 @@ private boolean isAllDependentTaskFinished() {
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 9f2b48357920c..b8ba05fe7aad8 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.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.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 DynamicAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext
this.subWorkflowService = subWorkflowService;
}
- @Override
- public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() {
+ public @NonNull TaskExecutionStatus getAsyncTaskExecutionStatus() {
List allSubWorkflowInstance = getAllSubProcessInstance();
int totalSubProcessInstanceCount = allSubWorkflowInstance.size();
@@ -90,16 +89,16 @@ public DynamicAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext
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 DynamicAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext
startSubProcessInstances(allSubWorkflowInstance, startCount);
}
// query the status of sub workflow instance
- return AsyncTaskExecutionStatus.RUNNING;
+ return TaskExecutionStatus.RUNNING_EXECUTION;
}
private void setOutputParameters() {
@@ -143,7 +142,7 @@ private void setOutputParameters() {
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 List getAllSubProcessInstance() {
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 567cd7ff88a02..2dc76c9e72255 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 6942260d8bbc7..40fd7ab69bc9f 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.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.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 DynamicLogicTask(TaskExecutionContext taskExecutionContext,
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 DynamicLogicTask(TaskExecutionContext taskExecutionContext,
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 4618c5ad59b6a..90ae7f12c3e29 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.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.PhysicalTaskEngine;
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 PhysicalTaskEngine physicalTaskEngine;
/**
* worker server startup, not use web service
@@ -86,14 +77,19 @@ public static void main(String[] args) {
@PostConstruct
public void run() {
ServerLifeCycleManager.toRunning();
+
this.workerRpcServer.start();
+
TaskPluginManager.loadTaskPlugin();
+
DataSourceProcessorProvider.initialize();
this.workerRegistryClient.setRegistryStoppable(this);
this.workerRegistryClient.start();
- this.messageRetryRunner.start();
+ // this.messageRetryRunner.start();
+
+ this.physicalTaskEngine.start();
WorkerServerMetrics.registerWorkerCpuUsageGauge(() -> {
SystemMetrics systemMetrics = metricsProvider.getSystemMetrics();
@@ -126,17 +122,10 @@ public void close(String cause) {
ThreadUtils.sleep(Constants.SERVER_CLOSE_WAIT_TIME.toMillis());
try (
+ PhysicalTaskEngine closedPhysicalTaskEngine = physicalTaskEngine;
WorkerRpcServer closedWorkerRpcServer = workerRpcServer;
WorkerRegistryClient closedRegistryClient = 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 +142,4 @@ public void stop(String cause) {
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 0000000000000..2cb9a120e02b9
--- /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 taskExecutorThreadCount = Runtime.getRuntime().availableProcessors() * 2;
+
+}
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 d901effbd1dbf..559b65dc3c139 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
@@ -58,6 +58,8 @@ public class WorkerConfig implements Validator {
private TenantConfig tenantConfig = new TenantConfig();
+ private PhysicalTaskConfig physicalTaskConfig = new PhysicalTaskConfig();
+
@Override
public boolean supports(Class> clazz) {
return WorkerConfig.class.isAssignableFrom(clazz);
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngine.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngine.java
new file mode 100644
index 0000000000000..98254a4cab7a5
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngine.java
@@ -0,0 +1,90 @@
+/*
+ * 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.ITaskExecutorLifecycleEventReporter;
+import org.apache.dolphinscheduler.task.executor.TaskEngine;
+import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterRequest;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.stereotype.Component;
+
+@Slf4j
+@Component
+public class PhysicalTaskEngine implements AutoCloseable {
+
+ private final TaskEngine taskEngine;
+
+ private final PhysicalTaskExecutorFactory physicalTaskExecutorFactory;
+
+ private final PhysicalTaskExecutorLifecycleEventReporter physicalTaskExecutorEventReporter;
+
+ private final PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
+
+ public PhysicalTaskEngine(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("PhysicalTaskEngine 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?
+ final ITaskExecutor iTaskExecutor = physicalTaskExecutorRepository.get(taskInstanceId);
+ if (iTaskExecutor != null) {
+ iTaskExecutor.getTaskExecutionContext().setWorkflowInstanceHost(workflowHost);
+ }
+ return physicalTaskExecutorEventReporter.reassignWorkflowInstanceHost(taskInstanceId, workflowHost);
+ }
+
+ @Override
+ public void close() {
+ taskEngine.close();
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutorFactory.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java
similarity index 50%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutorFactory.java
rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java
index b591711e3d02f..e25bfb57acf86 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutorFactory.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskEngineFactory.java
@@ -15,31 +15,33 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.runner.execute;
+package org.apache.dolphinscheduler.server.worker.executor;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager;
-import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
-
-import lombok.extern.slf4j.Slf4j;
+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;
-@Slf4j
@Component
-public class SyncMasterTaskExecutorFactory
- implements
- MasterTaskExecutorFactory {
+public class PhysicalTaskEngineFactory {
@Autowired
- private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
+ private PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
+
@Autowired
- private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager;
+ private PhysicalTaskExecutorContainerDelegator physicalTaskExecutorContainerDelegator;
- @Override
- public SyncMasterTaskExecutor createMasterTaskExecutor(TaskExecutionContext taskExecutionContext) {
- return new SyncMasterTaskExecutor(taskExecutionContext, logicTaskPluginFactoryBuilder,
- logicTaskInstanceExecutionEventSenderManager);
+ @Autowired
+ private PhysicalTaskExecutorEventBusCoordinator physicalTaskExecutorEventBusCoordinator;
+
+ public TaskEngine createTaskEngine() {
+ final TaskEngineBuilder taskEngineBuilder = TaskEngineBuilder.builder()
+ .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 0000000000000..d13f697dba784
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutor.java
@@ -0,0 +1,168 @@
+/*
+ * 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.TaskExecutionContext;
+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.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.TaskExecutorEventBus;
+import org.apache.dolphinscheduler.task.executor.TaskExecutorState;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorRuntimeContextChangedLifecycleEvent;
+import org.apache.dolphinscheduler.task.executor.event.TaskExecutorStartedLifecycleEvent;
+
+import lombok.Getter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class PhysicalTaskExecutor extends AbstractTaskExecutor {
+
+ private final WorkerConfig workerConfig;
+
+ private final StorageOperator storageOperator;
+
+ @Getter
+ private final TaskExecutionContext taskExecutionContext;
+
+ @Getter
+ private final TaskExecutorEventBus taskExecutorEventBus;
+
+ @Getter
+ private AbstractTask physicalTask;
+
+ private final PhysicalTaskPluginFactory physicalTaskPluginFactory;
+
+ public PhysicalTaskExecutor(final PhysicalTaskExecutorBuilder physicalTaskExecutorBuilder) {
+ super();
+ this.workerConfig = physicalTaskExecutorBuilder.getWorkerConfig();
+ this.storageOperator = physicalTaskExecutorBuilder.getStorageOperator();
+ this.taskExecutionContext = physicalTaskExecutorBuilder.getTaskExecutionContext();
+ this.physicalTaskPluginFactory = physicalTaskExecutorBuilder.getPhysicalTaskPluginFactory();
+ this.taskExecutorEventBus = physicalTaskExecutorBuilder.getTaskExecutorEventBus();
+ }
+
+ @SneakyThrows
+ @Override
+ public void start() {
+ // todo:从WorkerTaskExecutor中拷贝代码过来
+ TaskInstanceLogHeader.printInitializeTaskContextHeader();
+ initializeTaskContext();
+
+ // todo: Support dry run
+
+ publishTaskRunningEvent();
+
+ TaskInstanceLogHeader.printLoadTaskInstancePluginHeader();
+ initializeLogicTaskPlugin();
+
+ TaskInstanceLogHeader.printExecuteTaskHeader();
+
+ physicalTask.handle(new TaskCallBack() {
+
+ @Override
+ public void updateRemoteApplicationInfo(final int taskInstanceId, final ApplicationInfo applicationInfo) {
+ taskExecutionContext.setAppIds(applicationInfo.getAppIds());
+ taskExecutorEventBus
+ .publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(PhysicalTaskExecutor.this));
+ }
+
+ @Override
+ public void updateTaskInstanceInfo(final int taskInstanceId) {
+ taskExecutorEventBus
+ .publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(PhysicalTaskExecutor.this));
+ }
+ });
+
+ }
+
+ @Override
+ protected TaskExecutionStatus doTrackTaskPluginStatus() {
+ return physicalTask.getExitStatus();
+ }
+
+ @Override
+ public void pause() {
+ log.warn("The physical doesn't support pause");
+ }
+
+ @SneakyThrows
+ @Override
+ public void kill() {
+ // todo: 判断当前任务是否开始执行
+ if (physicalTask != null) {
+ physicalTask.cancel();
+ }
+ }
+
+ private void initializeTaskContext() {
+ log.info("Begin to initialize taskContext");
+ taskExecutionContext.setStartTime(System.currentTimeMillis());
+
+ 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));
+ }
+
+ private void publishTaskRunningEvent() {
+ transitTaskExecutorState(TaskExecutorState.RUNNING);
+ taskExecutorEventBus.publish(TaskExecutorStartedLifecycleEvent.of(this));
+ }
+
+ @SneakyThrows
+ private void initializeLogicTaskPlugin() {
+ this.physicalTask = physicalTaskPluginFactory.createPhysicalTask(this);
+ log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType());
+
+ this.physicalTask.init();
+
+ this.physicalTask.getParameters().setVarPool(taskExecutionContext.getVarPool());
+ log.info("Set taskVarPool: {} successfully", taskExecutionContext.getVarPool());
+
+ }
+
+}
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 0000000000000..6dfcdb675153d
--- /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.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/PhysicalTaskExecutorContainerDelegator.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorContainerDelegator.java
new file mode 100644
index 0000000000000..253da07483382
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorContainerDelegator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.PhysicalTaskConfig;
+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.ITaskExecutorContainerDelegator;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerBuilder;
+import org.apache.dolphinscheduler.task.executor.container.TaskExecutorContainerConfig;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorContainerDelegator implements ITaskExecutorContainerDelegator {
+
+ private final ITaskExecutorContainer taskExecutorContainer;
+
+ public PhysicalTaskExecutorContainerDelegator(final WorkerConfig masterConfig) {
+ final PhysicalTaskConfig physicalTaskConfig = masterConfig.getPhysicalTaskConfig();
+ final TaskExecutorContainerBuilder taskExecutorContainerBuilder = TaskExecutorContainerBuilder.builder()
+ .taskExecutorContainerName("ExclusiveThreadTaskExecutorContainer")
+ .taskExecutorContainerConfig(
+ new TaskExecutorContainerConfig(physicalTaskConfig.getTaskExecutorThreadCount()))
+ .build();
+ this.taskExecutorContainer = new ExclusiveThreadTaskExecutorContainer(taskExecutorContainerBuilder);
+ }
+
+ @Override
+ public ITaskExecutorContainer getExecutorContainer(final String taskType) {
+ return taskExecutorContainer;
+ }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java
new file mode 100644
index 0000000000000..776a928c94112
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorEventBusCoordinator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.TaskExecutorEventBusCoordinator;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorEventBusCoordinator extends TaskExecutorEventBusCoordinator {
+
+ public PhysicalTaskExecutorEventBusCoordinator(final PhysicalTaskExecutorRepository physicalTaskExecutorRepository,
+ final PhysicalTaskExecutorLifecycleEventListener physicalTaskExecutorLifecycleEventListener) {
+ super(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 0000000000000..18efd8839e304
--- /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 0000000000000..7330bddcc270f
--- /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 0000000000000..51ed1172c8096
--- /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 PhysicalTaskExecutorContainerDelegator 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 0000000000000..57d5aef06cc69
--- /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.TaskExecutorLifecycleEventRemoteReporter;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorLifecycleEventReporter extends TaskExecutorLifecycleEventRemoteReporter {
+
+ public PhysicalTaskExecutorLifecycleEventReporter(
+ final PhysicalTaskExecutorEventRemoteReporterClient physicalTaskExecutorEventRemoteReporterClient) {
+ super("PhysicalTaskExecutorLifecycleEventReporter", physicalTaskExecutorEventRemoteReporterClient);
+ }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java
new file mode 100644
index 0000000000000..751c01313a519
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskExecutorRepository.java
@@ -0,0 +1,61 @@
+/*
+ * 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 static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.task.executor.ITaskExecutor;
+import org.apache.dolphinscheduler.task.executor.ITaskExecutorRepository;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.springframework.stereotype.Component;
+
+@Component
+public class PhysicalTaskExecutorRepository implements ITaskExecutorRepository {
+
+ private final Map taskExecutorMap = new ConcurrentHashMap<>();
+
+ @Override
+ public void put(final ITaskExecutor taskExecutor) {
+ checkNotNull(taskExecutor);
+ taskExecutorMap.put(taskExecutor.getId(), taskExecutor);
+ }
+
+ @Override
+ public ITaskExecutor get(final Integer taskExecutorId) {
+ return taskExecutorMap.get(taskExecutorId);
+ }
+
+ @Override
+ public Collection getAll() {
+ return taskExecutorMap.values();
+ }
+
+ @Override
+ public boolean contains(final Integer taskExecutorId) {
+ return taskExecutorMap.containsKey(taskExecutorId);
+ }
+
+ @Override
+ public void remove(final Integer taskExecutorId) {
+ taskExecutorMap.remove(taskExecutorId);
+ }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
similarity index 52%
rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java
rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
index 51bda7edbee8a..b37fc9ecf7f1c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/executor/PhysicalTaskPluginFactory.java
@@ -15,32 +15,24 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.server.master.runner.execute;
+package org.apache.dolphinscheduler.server.worker.executor;
-import java.util.concurrent.DelayQueue;
-
-import javax.annotation.Nullable;
-
-import lombok.NonNull;
+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 org.springframework.stereotype.Component;
@Component
-public class AsyncMasterTaskDelayQueue {
-
- private final DelayQueue asyncTaskCheckDelayQueue = new DelayQueue<>();
+public class PhysicalTaskPluginFactory {
- public void addAsyncTask(@NonNull AsyncTaskExecutionContext asyncTaskExecutionContext) {
- asyncTaskExecutionContext.refreshStartTime();
- asyncTaskCheckDelayQueue.add(asyncTaskExecutionContext);
+ public AbstractTask createPhysicalTask(final PhysicalTaskExecutor physicalTaskExecutor) {
+ TaskChannel taskChannel = getTaskChannel(physicalTaskExecutor);
+ return taskChannel.createTask(physicalTaskExecutor.getTaskExecutionContext());
}
- public @Nullable AsyncTaskExecutionContext pollAsyncTask() throws InterruptedException {
- return asyncTaskCheckDelayQueue.take();
+ public TaskChannel getTaskChannel(final PhysicalTaskExecutor physicalTaskExecutor) {
+ final String taskType = physicalTaskExecutor.getTaskExecutionContext().getTaskType();
+ return TaskPluginManager.getTaskChannel(taskType);
}
-
- public int getAsyncTaskRunningNum() {
- return asyncTaskCheckDelayQueue.size();
- }
-
}
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 a4312e3cdb25c..0000000000000
--- 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 5b3d12f407ceb..0000000000000
--- 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 c3fa13550d46e..0000000000000
--- 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 e5222d8bc821c..0000000000000
--- 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 7e399542fbd48..0000000000000
--- 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/TaskExecutionSuccessEventSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionSuccessEventSender.java
deleted file mode 100644
index 967e0b9ebf462..0000000000000
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskExecutionSuccessEventSender.java
+++ /dev/null
@@ -1,59 +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.TaskExecutionSuccessEvent;
-import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-
-import org.springframework.stereotype.Component;
-
-@Component
-public class TaskExecutionSuccessEventSender
- implements
- TaskInstanceExecutionEventSender {
-
- @Override
- public void sendEvent(TaskExecutionSuccessEvent message) {
- Clients
- .withService(ITaskExecutionEventListener.class)
- .withHost(message.getWorkflowInstanceHost())
- .onTaskInstanceExecutionSuccess(message);
- }
-
- @Override
- public TaskExecutionSuccessEvent buildEvent(TaskExecutionContext taskExecutionContext) {
- return TaskExecutionSuccessEvent.builder()
- .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId())
- .taskInstanceId(taskExecutionContext.getTaskInstanceId())
- .appIds(taskExecutionContext.getAppIds())
- .processId(taskExecutionContext.getProcessId())
- .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost())
- .taskInstanceHost(taskExecutionContext.getHost())
- .endTime(taskExecutionContext.getEndTime())
- .varPool(taskExecutionContext.getVarPool())
- .build();
- }
-
- @Override
- public ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType() {
- return ITaskExecutionEvent.TaskInstanceExecutionEventType.SUCCESS;
- }
-}
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 53581e7abd96f..0000000000000
--- 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/rpc/PhysicalTaskExecutorOperatorImpl.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/PhysicalTaskExecutorOperatorImpl.java
new file mode 100644
index 0000000000000..69fea9050b623
--- /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.PhysicalTaskEngine;
+import org.apache.dolphinscheduler.task.executor.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 PhysicalTaskEngine physicalTaskEngine;
+
+ @Override
+ public TaskExecutorDispatchResponse dispatchTask(final TaskExecutorDispatchRequest taskExecutorDispatchRequest) {
+ log.info("Receive TaskExecutorDispatchResponse: {}", taskExecutorDispatchRequest);
+ final TaskExecutionContext taskExecutionContext = taskExecutorDispatchRequest.getTaskExecutionContext();
+ try {
+ physicalTaskEngine.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 {
+ physicalTaskEngine.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 {
+ physicalTaskEngine.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 =
+ physicalTaskEngine.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);
+ physicalTaskEngine.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 51b75aa457166..18c308686d594 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,8 @@
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 lombok.extern.slf4j.Slf4j;
@@ -37,7 +36,7 @@
public class StreamingTaskInstanceOperatorImpl implements IStreamingTaskInstanceOperator {
@Autowired
- private WorkerTaskExecutorThreadPool workerManager;
+ private PhysicalTaskExecutorRepository physicalTaskExecutorRepository;
@Override
public TaskInstanceTriggerSavepointResponse triggerSavepoint(TaskInstanceTriggerSavepointRequest taskInstanceTriggerSavepointRequest) {
@@ -46,12 +45,13 @@ public TaskInstanceTriggerSavepointResponse triggerSavepoint(TaskInstanceTrigger
try {
int taskInstanceId = taskInstanceTriggerSavepointRequest.getTaskInstanceId();
LogUtils.setTaskInstanceIdMDC(taskInstanceId);
- WorkerTaskExecutor workerTaskExecutor = WorkerTaskExecutorHolder.get(taskInstanceId);
- if (workerTaskExecutor == null) {
+ final PhysicalTaskExecutor taskExecutor =
+ (PhysicalTaskExecutor) physicalTaskExecutorRepository.get(taskInstanceId);
+ if (taskExecutor == null) {
log.error("Cannot find WorkerTaskExecutor for taskInstance: {}", taskInstanceId);
return TaskInstanceTriggerSavepointResponse.fail("Cannot find TaskExecutionContext");
}
- AbstractTask task = workerTaskExecutor.getTask();
+ 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 c086a77bfe199..0000000000000
--- 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 d03df8a88c8bd..0000000000000
--- 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 a8a0c5a2b7ffd..b1bca6dfe67ff 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 TaskInstanceLogPageQueryResponse pageQueryTaskInstanceLog(TaskInstanceLog
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 36c1345662de5..0000000000000
--- 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 f5abf9e577ff4..0000000000000
--- 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 085deafb09f67..0000000000000
--- 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 c05fab390b1c8..0000000000000
--- 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 d519b1420e125..0000000000000
--- 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 4cb1739abcbc4..0000000000000
--- 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 e44b23152cc6f..0000000000000
--- 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 d4b020639459f..0000000000000
--- 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 bc5d612b66031..0000000000000
--- 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 4cc40ada6c420..0000000000000
--- 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 fe114ae6c92d2..0000000000000
--- 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 a32ced01ab5db..0000000000000
--- 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 233f215e59e60..0000000000000
--- 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/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/runner/DefaultWorkerTaskExecutorTest.java
index 8b527fe5b3dfe..f1c1ecf143347 100644
--- 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
@@ -23,7 +23,6 @@
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;
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
index f33bf1faae1d1..49be58060a393 100644
--- 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
@@ -23,7 +23,6 @@
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;
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 a8edf4d5a68be..0000000000000
--- 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 80560f8334b93..70e7b94a7e41c 100755
--- a/pom.xml
+++ b/pom.xml
@@ -59,6 +59,7 @@
dolphinscheduler-dao-plugin
dolphinscheduler-authentication
dolphinscheduler-eventbus
+ dolphinscheduler-task-executor